From f2f1b2cedfe58165c5443de17bc97174f1969fa1 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Mon, 16 Dec 2024 13:11:20 +0000 Subject: [PATCH 001/126] roachprod: use logger for monitor CLI Previously, running roachprod Monitor, from CLI, did not include a timestamp when an event is fired. Using a logger adds a timestamp to the output which makes it more useful to determine when a monitor event happened. Epic: None # Conflicts: # pkg/cmd/roachprod/main.go --- pkg/cmd/roachprod/cli/commands.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cmd/roachprod/cli/commands.go b/pkg/cmd/roachprod/cli/commands.go index 80cccc36a574..e0dbf5d683cb 100644 --- a/pkg/cmd/roachprod/cli/commands.go +++ b/pkg/cmd/roachprod/cli/commands.go @@ -672,7 +672,7 @@ of nodes, outputting a line whenever a change is detected: return err } for info := range eventChan { - fmt.Println(info.String()) + config.Logger.Printf(info.String()) } return nil From a09069f29564ab75b47ec63eb0ca869aaf8a9595 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Tue, 17 Dec 2024 15:01:22 +0000 Subject: [PATCH 002/126] roachprod: monitor local script Add a new monitor script for monitoring local processes. On OSX development machines `systemctl` is not available so we need to rely on process status, `ps` command, to determine which processes are running. One limitation of using `ps` is that we are unable to determine the status for a process that has exited. But to keep consistent with the output Monitor expects we append a status line "unknown" for each process. The biggest difference between the new and old script is that this script does not contain any logic to detect process changes, but rather only provides information on cockroach processes back to the caller. It does however optimise a little bit by not resending the information if it has not changed. This now puts the responsibility on the caller (Monitor) to maintain the logic for detecting process changes and emitting events. Epic: None --- .../install/scripts/monitor_local.sh | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) create mode 100644 pkg/roachprod/install/scripts/monitor_local.sh diff --git a/pkg/roachprod/install/scripts/monitor_local.sh b/pkg/roachprod/install/scripts/monitor_local.sh new file mode 100644 index 000000000000..5c465c72196b --- /dev/null +++ b/pkg/roachprod/install/scripts/monitor_local.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash +# +# Copyright 2024 The Cockroach Authors. +# +# Use of this software is governed by the CockroachDB Software License +# included in the /LICENSE file. + +# This script is used to monitor the status of cockroach processes on local nodes +# using `ps`. It does not support checking the exit status of a process, but +# still outputs the status line for consistency with the remote monitor. +# It produces output in the following format: +#cockroach-system=500 +#status=unknown +#cockroach-tenant_0=501 +#status=unknown +#\n = end of frame + +roachprod_regex=#{shesc .RoachprodEnvRegex#} +one_shot=#{if .OneShot#}true#{end#} + +prev_frame="" +while :; do + # Get the PID and command of all processes that match the roachprod regex. + ps_output=$(ps axeww -o pid,command | grep -v grep | grep -E "$roachprod_regex") + frame="" + while IFS= read -r line; do + # Extract the PID and command from the line. + read -r pid command <<< "$line" + # If the command contains the `--background` flag, skip it. It's the command + # that is starting the background cockroach process. + if [[ "$command" == *"--background"* ]]; then + continue + fi + # Extract the virtual cluster label from the command. + vc_label=$(echo "$command" | grep -E -o 'ROACHPROD_VIRTUAL_CLUSTER=[^ ]*' | cut -d= -f2) + # If the virtual cluster label is not empty, print the label and the PID. + # Also print the status of the process, if remote, where systemd is available. + if [ -n "$vc_label" ]; then + frame+="$vc_label=$pid\n" + # If the process is local we can't check the status (exit code). + frame+="status=unknown\n" + fi + done <<< "$ps_output" + # Only print the frame if it has changed. + if [ "$frame" != "$prev_frame" ]; then + echo -e "$frame" + prev_frame="$frame" + fi + # If one_shot is set, exit after the first iteration. + if [[ -n "${one_shot}" ]]; then + break + fi + sleep 1 +done From 78a873fd4c93dee03d64c510bf3e99c89bd70d86 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Wed, 18 Dec 2024 09:42:32 +0000 Subject: [PATCH 003/126] roachprod: monitor remote script Add a new monitor script for monitoring remote processes. `systemctl` is used to monitor cockroach processes, on remote nodes, and determine the exit status if a process has died. It emits the same frame format as the local script version, and depends on Monitor to implement the logic for detecting changes in the process list frame. Informs: #118214 Epic: None --- .../install/scripts/monitor_remote.sh | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) create mode 100755 pkg/roachprod/install/scripts/monitor_remote.sh diff --git a/pkg/roachprod/install/scripts/monitor_remote.sh b/pkg/roachprod/install/scripts/monitor_remote.sh new file mode 100755 index 000000000000..2f76593a3a48 --- /dev/null +++ b/pkg/roachprod/install/scripts/monitor_remote.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# +# Copyright 2024 The Cockroach Authors. +# +# Use of this software is governed by the CockroachDB Software License +# included in the /LICENSE file. + +# This script is used to monitor the status of cockroach processes on a remote +# node where systemctl is available. +# It produces output in the following format: +#cockroach-system=500 +#status=unknown +#cockroach-tenant_0=501 +#status=1 +#\n = end of frame + +one_shot=#{if .OneShot#}true#{end#} + +prev_frame="" +while :; do + # Get all cockroach system units + sysctl_output=$(systemctl list-units cockroach\*.service --type=service --no-legend --no-pager | awk '{print $1}') + frame="" + while IFS= read -r name; do + # Query the PID and status of the cockroach system unit + pid=$(systemctl show "$name" --property MainPID --value) + status=$(systemctl show "$name" --property ExecMainStatus --value) + vc_label=${name%.service} + frame+="$vc_label=$pid\n" + frame+="status=$status\n" + done <<< "$sysctl_output" + # Only print the frame if it has changed. + if [ "$frame" != "$prev_frame" ]; then + echo -e "$frame" + prev_frame="$frame" + fi + # If one_shot is set, exit after the first iteration. + if [[ -n "${one_shot}" ]]; then + break + fi + sleep 1 +done From b8371641b129db4d2bf1efd31c2fff2d071fcda5 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Wed, 18 Dec 2024 09:52:12 +0000 Subject: [PATCH 004/126] roachprod: mock session Previously, it was difficult to test code that creates a remote session. This change adds a mock implementation of session, and the hooks in SyncedCluster to use it during tests. The mock session provides several hooks for controlling the IO of a session. The inputs and outputs are backed by a channel to allow finer control of the flow of data. Currently, it only implements what was required to test the Monitor functionality, but can be extended in the future to cover more use cases. Informs: #118214 Epic: None --- pkg/roachprod/install/BUILD.bazel | 1 + pkg/roachprod/install/cluster_synced.go | 7 ++ pkg/roachprod/install/session_test.go | 142 ++++++++++++++++++++++++ 3 files changed, 150 insertions(+) create mode 100644 pkg/roachprod/install/session_test.go diff --git a/pkg/roachprod/install/BUILD.bazel b/pkg/roachprod/install/BUILD.bazel index 3d6390bdc97d..fb3a06edec1f 100644 --- a/pkg/roachprod/install/BUILD.bazel +++ b/pkg/roachprod/install/BUILD.bazel @@ -56,6 +56,7 @@ go_test( "cluster_synced_test.go", "cockroach_test.go", "services_test.go", + "session_test.go", "staging_test.go", "start_template_test.go", ], diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index 11545557727c..dd074203dccb 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -81,6 +81,10 @@ type SyncedCluster struct { // AuthorizedKeys is used by SetupSSH to add additional authorized keys. AuthorizedKeys []byte + + // sessionProvider is a function that returns a new session. It serves as a + // testing hook, and if null the default session implementations will be used. + sessionProvider func(node Node, cmd string) session } // NewSyncedCluster creates a SyncedCluster, given the cluster metadata, node @@ -385,6 +389,9 @@ func (c *SyncedCluster) validateHost(ctx context.Context, l *logger.Logger, node func (c *SyncedCluster) newSession( l *logger.Logger, node Node, cmd string, options ...remoteSessionOption, ) session { + if c.sessionProvider != nil { + return c.sessionProvider(node, cmd) + } if c.IsLocal() { return newLocalSession(cmd) } diff --git a/pkg/roachprod/install/session_test.go b/pkg/roachprod/install/session_test.go new file mode 100644 index 000000000000..85bf8be2cc07 --- /dev/null +++ b/pkg/roachprod/install/session_test.go @@ -0,0 +1,142 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package install + +import ( + "context" + "io" + "sync/atomic" +) + +type MockSession struct { + OnCombinedOutput func(context.Context) ([]byte, error) + OnRun func(context.Context) error + OnStart func(context.Context) error + OnWait func() error + + Stdout io.ReadWriteCloser + Stderr io.ReadWriteCloser +} + +type MockSessionOptions struct { + ChannelSize int +} + +// dataChannel emulates blocking reads and writes with a channel. +type dataChannel struct { + dataChan chan []byte + closed atomic.Bool +} + +// MockSession implements the session interface. +var _ session = &MockSession{} + +func (m MockSession) CombinedOutput(ctx context.Context) ([]byte, error) { + if m.OnCombinedOutput != nil { + return m.OnCombinedOutput(ctx) + } + return nil, nil +} + +func (m MockSession) Run(ctx context.Context) error { + if m.OnRun != nil { + return m.OnRun(ctx) + } + return nil +} + +func (m MockSession) Start() error { + if m.OnStart != nil { + return m.OnStart(context.Background()) + } + return nil +} + +func (m MockSession) RequestPty() error { + return nil +} + +func (m MockSession) Wait() error { + if m.OnWait != nil { + return m.OnWait() + } + return nil +} + +func (m MockSession) Close() { + _ = m.Stdout.Close() + _ = m.Stderr.Close() +} + +func (m MockSession) SetStdin(_ io.Reader) { + panic("not implemented") +} + +func (m MockSession) SetStdout(_ io.Writer) { + panic("not implemented") +} + +func (m MockSession) SetStderr(_ io.Writer) { + panic("not implemented") +} + +func (m MockSession) StdinPipe() (io.WriteCloser, error) { + panic("not implemented") +} + +func (m MockSession) StdoutPipe() (io.Reader, error) { + return m.Stdout, nil +} + +func (m MockSession) StderrPipe() (io.Reader, error) { + return m.Stderr, nil +} + +// NewMockSession creates a new mock session. The mock session can be used to +// simulate the Stdout and Stderr of a session. Stdout and Stderr are backed by +// channels, that are unbuffered by default. +func NewMockSession(opts MockSessionOptions) *MockSession { + return &MockSession{ + Stdout: newDataChannel(opts.ChannelSize), + Stderr: newDataChannel(opts.ChannelSize), + } +} + +// DefaultMockSessionOptions returns the default options for a mock session. +func DefaultMockSessionOptions() MockSessionOptions { + return MockSessionOptions{} +} + +func newDataChannel(channelSize int) io.ReadWriteCloser { + return &dataChannel{dataChan: make(chan []byte, channelSize)} +} + +// Close is a no-op if the channel is already closed. This allows tests or the +// implementation to close the channel without panicking. +func (b *dataChannel) Close() error { + if b.closed.Swap(true) { + return nil + } + close(b.dataChan) + return nil +} + +func (b *dataChannel) Write(p []byte) (n int, err error) { + b.dataChan <- p + return len(p), nil +} + +func (b *dataChannel) Read(p []byte) (n int, err error) { + data := <-b.dataChan + if data == nil { + return 0, io.EOF + } + copiedLen := copy(p, data) + if copiedLen < len(data) { + b.dataChan <- data[copiedLen:] + } + return copiedLen, nil +} From 1b1b1b7632b5771da4c660b73afa1ab6c05cea49 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Thu, 2 Jan 2025 11:12:22 +0000 Subject: [PATCH 005/126] roachprod: refactor Monitor Previously, Monitor was only able to monitor processes already started. It could not detect a new processes if it was started after monitor started monitoring. This change uses the new monitor scripts `monitor_local.sh` & `monitor_remote.sh` which produces a frame of processes when any cockroach process changes occur including when a new cockroach process is started. The logic for detecting changes in processes has been moved to the "client" side. The scripts are now less complicated and only have the responsibility of sending through a frame of processes. A frame consists of the cluster label, process id, and processes status of all cockroach processes. In addition, Monitor has been moved into its own source file for better separation of logic. The `ignore empty nodes` functionality has also been removed, as it is an unreliable way of detecting "empty nodes", and does not serve any real purpose anymore. Informs: #118214 Epic: None --- pkg/cmd/roachtest/cluster.go | 4 +- pkg/roachprod/install/BUILD.bazel | 3 + pkg/roachprod/install/cluster_synced.go | 429 ------------------------ pkg/roachprod/install/monitor.go | 386 +++++++++++++++++++++ 4 files changed, 391 insertions(+), 431 deletions(-) create mode 100644 pkg/roachprod/install/monitor.go diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index d9801a212c2c..de1cfbb15ced 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1480,7 +1480,7 @@ func (c *clusterImpl) FetchVMSpecs(ctx context.Context, l *logger.Logger) error }) } -// checkNoDeadNode returns an error if at least one of the nodes that have a populated +// assertNoDeadNode returns an error if at least one of the nodes that have a populated // data dir are found to be not running. It prints both to t.L() and the test // output. func (c *clusterImpl) assertNoDeadNode(ctx context.Context, t test.Test) error { @@ -1490,7 +1490,7 @@ func (c *clusterImpl) assertNoDeadNode(ctx context.Context, t test.Test) error { } t.L().Printf("checking for dead nodes") - eventsCh, err := roachprod.Monitor(ctx, t.L(), c.name, install.MonitorOpts{OneShot: true, IgnoreEmptyNodes: true}) + eventsCh, err := roachprod.Monitor(ctx, t.L(), c.name, install.MonitorOpts{OneShot: true}) // An error here means there was a problem initialising a SyncedCluster. if err != nil { diff --git a/pkg/roachprod/install/BUILD.bazel b/pkg/roachprod/install/BUILD.bazel index fb3a06edec1f..60c59fda0a2f 100644 --- a/pkg/roachprod/install/BUILD.bazel +++ b/pkg/roachprod/install/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "expander.go", "install.go", "iterm2.go", + "monitor.go", "nodes.go", "run_options.go", "services.go", @@ -21,6 +22,8 @@ go_library( "scripts/download.sh", "scripts/start.sh", "scripts/open_ports.sh", + "scripts/monitor_remote.sh", + "scripts/monitor_local.sh", ], importpath = "github.com/cockroachdb/cockroach/pkg/roachprod/install", visibility = ["//visibility:public"], diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index dd074203dccb..82a19588cb93 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -6,7 +6,6 @@ package install import ( - "bufio" "bytes" "context" "fmt" @@ -17,11 +16,9 @@ import ( "os/exec" "os/signal" "path/filepath" - "strconv" "strings" "sync" "syscall" - "text/template" "time" "github.com/cockroachdb/cockroach/pkg/roachprod/cloud" @@ -679,432 +676,6 @@ fi return statuses, nil } -// MonitorProcessSkipped represents a cockroach process whose status -// was not checked. -type MonitorProcessSkipped struct { - VirtualClusterName string - SQLInstance int -} - -// MonitorProcessRunning represents the cockroach process running on a -// node. -type MonitorProcessRunning struct { - VirtualClusterName string - SQLInstance int - PID string -} - -// MonitorProcessDead represents the cockroach process dying on a node. -type MonitorProcessDead struct { - VirtualClusterName string - SQLInstance int - ExitCode string -} - -type MonitorError struct { - Err error -} - -// MonitorNoCockroachProcessesError is the error returned when the -// monitor is called on a node that is not running a `cockroach` -// process by the time the monitor runs. -var MonitorNoCockroachProcessesError = errors.New("no cockroach processes running") - -// NodeMonitorInfo is a message describing a cockroach process' status. -type NodeMonitorInfo struct { - // The index of the node (in a SyncedCluster) at which the message originated. - Node Node - // Event describes what happened to the node; it is one of - // MonitorProcessSkipped (no store directory was found); - // MonitorProcessRunning, sent when cockroach is running on a node; - // MonitorProcessDead, when the cockroach process stops running on a - // node; or MonitorError, typically indicate networking issues or - // nodes that have (physically) shut down. - Event interface{} -} - -func (nmi NodeMonitorInfo) String() string { - var status string - - virtualClusterDesc := func(name string, instance int) string { - if name == SystemInterfaceName { - return "system interface" - } - - return fmt.Sprintf("virtual cluster %q, instance %d", name, instance) - } - - switch event := nmi.Event.(type) { - case MonitorProcessRunning: - status = fmt.Sprintf("cockroach process for %s is running (PID: %s)", - virtualClusterDesc(event.VirtualClusterName, event.SQLInstance), event.PID, - ) - case MonitorProcessSkipped: - status = fmt.Sprintf("%s was skipped", virtualClusterDesc(event.VirtualClusterName, event.SQLInstance)) - case MonitorProcessDead: - status = fmt.Sprintf("cockroach process for %s died (exit code %s)", - virtualClusterDesc(event.VirtualClusterName, event.SQLInstance), event.ExitCode, - ) - case MonitorError: - status = fmt.Sprintf("error: %s", event.Err.Error()) - } - - return fmt.Sprintf("n%d: %s", nmi.Node, status) -} - -// MonitorOpts is used to pass the options needed by Monitor. -type MonitorOpts struct { - OneShot bool // Report the status of all targeted nodes once, then exit. - IgnoreEmptyNodes bool // Only monitor nodes with a nontrivial data directory. -} - -// Monitor writes NodeMonitorInfo for the cluster nodes to the returned channel. -// Infos sent to the channel always have the Node the event refers to, and the -// event itself. See documentation for NodeMonitorInfo for possible event types. -// -// If OneShot is true, infos are retrieved only once for each node and the -// channel is subsequently closed; otherwise the process continues indefinitely -// (emitting new information as the status of the cockroach process changes). -// -// If IgnoreEmptyNodes is true, tenants on which no CockroachDB data is found -// (in {store-dir}) will not be probed and single event, MonitorTenantSkipped, -// will be emitted for each tenant. -// -// Note that the monitor will only send events for tenants that exist -// at the time this function is called. In other words, this function -// will not emit events for tenants started *after* a call to -// Monitor(). -func (c *SyncedCluster) Monitor( - l *logger.Logger, ctx context.Context, opts MonitorOpts, -) chan NodeMonitorInfo { - ch := make(chan NodeMonitorInfo) - nodes := c.TargetNodes() - var wg sync.WaitGroup - monitorCtx, cancel := context.WithCancel(ctx) - - // sendEvent sends the NodeMonitorInfo passed through the channel - // that is listened to by the caller. Bails if the context is - // canceled. - sendEvent := func(info NodeMonitorInfo) { - // if the monitor's context is already canceled, do not attempt to - // send the error down the channel, as it is most likely *caused* - // by the cancelation itself. - if monitorCtx.Err() != nil { - return - } - - select { - case ch <- info: - // We were able to send the info through the channel. - case <-monitorCtx.Done(): - // Don't block trying to send the info. - } - } - - const ( - separator = "|" - skippedMsg = "skipped" - runningMsg = "running" - deadMsg = "dead" - ) - - wg.Add(len(nodes)) - for i := range nodes { - go func(i int) { - defer wg.Done() - node := nodes[i] - - // We first find out all cockroach processes that are currently - // running in this node. - cockroachProcessesCmd := fmt.Sprintf(`ps axeww -o command | `+ - `grep -E '%s' | `+ // processes started by roachprod - `grep -E -o 'ROACHPROD_VIRTUAL_CLUSTER=[^ ]*' | `+ // ROACHPROD_VIRTUAL_CLUSTER indicates this is a cockroach process - `cut -d= -f2`, // grab the virtual cluster label - c.roachprodEnvRegex(node), - ) - - result, err := c.runCmdOnSingleNode( - ctx, l, node, cockroachProcessesCmd, defaultCmdOpts("list-processes"), - ) - if err := errors.CombineErrors(err, result.Err); err != nil { - err := errors.Wrap(err, "failed to list cockroach processes") - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - return - } - - type virtualClusterInfo struct { - Name string - Instance int - } - - // Make the collection of virtual clusters a set to handle the - // unlikely but possible case that, in `local` runs, we'll find - // two processes associated with the same virtual cluster - // label. This can happen if we invoke the command above while the - // parent cockroach process already created the child, - // background process, but has not terminated yet. - vcs := map[virtualClusterInfo]struct{}{} - vcLines := strings.TrimSuffix(result.CombinedOut, "\n") - if vcLines == "" { - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{MonitorNoCockroachProcessesError}}) - return - } - for _, label := range strings.Split(vcLines, "\n") { - name, instance, err := VirtualClusterInfoFromLabel(label) - if err != nil { - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - return - } - vcs[virtualClusterInfo{name, instance}] = struct{}{} - } - - data := struct { - OneShot bool - Node Node - IgnoreEmpty bool - Store string - Local bool - Separator string - SkippedMsg string - RunningMsg string - DeadMsg string - Processes []virtualClusterInfo - }{ - OneShot: opts.OneShot, - Node: node, - IgnoreEmpty: opts.IgnoreEmptyNodes, - Store: c.NodeDir(node, 1 /* storeIndex */), - Local: c.IsLocal(), - Separator: separator, - SkippedMsg: skippedMsg, - RunningMsg: runningMsg, - DeadMsg: deadMsg, - Processes: maps.Keys(vcs), - } - - storeFor := func(name string, instance int) string { - return c.InstanceStoreDir(node, name, instance) - } - - localPIDFile := func(name string, instance int) string { - return filepath.Join(c.LogDir(node, name, instance), "cockroach.pid") - } - - // NB.: we parse the output of every line this script - // prints. Every call to `echo` must match the parsing logic - // down below in order to produce structured results to the - // caller. - snippet := ` -dead_parent() { - ! ps -p "$1" >/dev/null || ps -o ucomm -p "$1" | grep -q defunct -} -{{ range .Processes }} -monitor_process_{{$.Node}}_{{.Name}}_{{.Instance}}() { - {{ if $.IgnoreEmpty }} - if ! ls {{storeFor .Name .Instance}}/marker.* 1> /dev/null 2>&1; then - echo "{{.Name}}{{$.Separator}}{{.Instance}}{{$.Separator}}{{$.SkippedMsg}}" - return 0 - fi - {{- end}} - # Init with -1 so that when cockroach is initially dead, we print - # a dead event for it. - lastpid=-1 - while :; do - # if parent process terminated, quit as well. - if dead_parent "$1"; then - return 0 - fi - {{ if $.Local }} - pidFile=$(cat "{{pidFile .Name .Instance}}") - # Make sure the process is still running - pid=$(test -n "${pidFile}" && ps -p "${pidFile}" >/dev/null && echo "${pidFile}") - pid=${pid:-0} # default to 0 - status="unknown" - {{- else }} - # When CRDB is not running, this is zero. - pid=$(systemctl show "{{virtualClusterLabel .Name .Instance}}" --property MainPID --value) - status=$(systemctl show "{{virtualClusterLabel .Name .Instance}}" --property ExecMainStatus --value) - {{- end }} - if [[ "${lastpid}" == -1 && "${pid}" != 0 ]]; then - # On the first iteration through the loop, if the process is running, - # don't register a PID change (which would trigger an erroneous dead - # event). - lastpid=0 - fi - # Output a dead event whenever the PID changes from a nonzero value to - # any other value. In particular, we emit a dead event when the node stops - # (lastpid is nonzero, pid is zero), but not when the process then starts - # again (lastpid is zero, pid is nonzero). - if [ "${pid}" != "${lastpid}" ]; then - if [ "${lastpid}" != 0 ]; then - if [ "${pid}" != 0 ]; then - # If the PID changed but neither is zero, then the status refers to - # the new incarnation. We lost the actual exit status of the old PID. - status="unknown" - fi - echo "{{.Name}}{{$.Separator}}{{.Instance}}{{$.Separator}}{{$.DeadMsg}}{{$.Separator}}${status}" - fi - if [ "${pid}" != 0 ]; then - echo "{{.Name}}{{$.Separator}}{{.Instance}}{{$.Separator}}{{$.RunningMsg}}{{$.Separator}}${pid}" - fi - lastpid=${pid} - fi - {{ if $.OneShot }} - return 0 - {{- end }} - sleep 1 - if [ "${pid}" != 0 ]; then - while kill -0 "${pid}" && ! dead_parent "$1"; do - sleep 1 - done - fi - done -} -{{ end }} - -# monitor every cockroach process in parallel. -{{ range .Processes }} -monitor_process_{{$.Node}}_{{.Name}}_{{.Instance}} $$ & -{{ end }} - -wait -` - - t := template.Must(template.New("script").Funcs(template.FuncMap{ - "storeFor": storeFor, - "pidFile": localPIDFile, - "virtualClusterLabel": VirtualClusterLabel, - }).Parse(snippet)) - var buf bytes.Buffer - if err := t.Execute(&buf, data); err != nil { - err := errors.Wrap(err, "failed to execute template") - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - return - } - - // This is the exception to funneling all SSH traffic through `c.runCmdOnSingleNode` - sess := c.newSession(l, node, buf.String(), withDebugDisabled()) - defer sess.Close() - - p, err := sess.StdoutPipe() - if err != nil { - err := errors.Wrap(err, "failed to read stdout pipe") - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - return - } - // Request a PTY so that the script will receive a SIGPIPE when the - // session is closed. - if err := sess.RequestPty(); err != nil { - err := errors.Wrap(err, "failed to request PTY") - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - return - } - - var readerWg sync.WaitGroup - readerWg.Add(1) - go func(p io.Reader) { - defer readerWg.Done() - r := bufio.NewReader(p) - for { - line, _, err := r.ReadLine() - if err == io.EOF { - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - return - } - if err != nil { - err := errors.Wrap(err, "error reading from session") - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - return - } - - parts := strings.Split(string(line), separator) - ensureNumParts := func(n int) { - if len(parts) < n { - panic(fmt.Errorf("invalid output from monitor: %q", line)) - } - } - // Every event is expected to have at least 3 parts. If - // that's not the case, panic explicitly below. Otherwise, - // we'd get a slice out of bounds error and the error - // message would not include the actual problematic line, - // which would make understanding the failure more - // difficult. - ensureNumParts(3) // name, instance, event - - // Virtual cluster name and instance are the first fields of - // every event type. - name, instanceStr := parts[0], parts[1] - instance, _ := strconv.Atoi(instanceStr) - switch parts[2] { - case skippedMsg: - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorProcessSkipped{ - VirtualClusterName: name, SQLInstance: instance, - }}) - case runningMsg: - ensureNumParts(4) - pid := parts[3] - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorProcessRunning{ - VirtualClusterName: name, SQLInstance: instance, PID: pid, - }}) - case deadMsg: - ensureNumParts(4) - exitCode := parts[3] - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorProcessDead{ - VirtualClusterName: name, SQLInstance: instance, ExitCode: exitCode, - }}) - default: - err := fmt.Errorf("internal error: unrecognized output from monitor: %q", line) - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - } - } - }(p) - - if err := sess.Start(); err != nil { - err := errors.Wrap(err, "failed to start session") - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - return - } - - // Watch for context cancellation, which can happen if the test - // fails, or if the monitor loop exits. - go func() { - <-monitorCtx.Done() - sess.Close() - if pc, ok := p.(io.ReadCloser); ok { - _ = pc.Close() - } - }() - - readerWg.Wait() - // We must call `sess.Wait()` only after finishing reading from the stdout - // pipe. Otherwise it can be closed under us, causing the reader to loop - // infinitely receiving a non-`io.EOF` error. - if err := sess.Wait(); err != nil { - // If we got an error waiting for the session but the context - // is already canceled, do not send an error through the - // channel; context cancelation happens at the user's request - // or when the test finishes. In either case, the monitor - // should quiesce. Reporting the error is confusing and can be - // spammy in the case of multiple monitors. - if monitorCtx.Err() != nil { - return - } - - err := errors.Wrap(err, "failed to wait for session") - sendEvent(NodeMonitorInfo{Node: node, Event: MonitorError{err}}) - return - } - }(i) - } - go func() { - wg.Wait() - cancel() - close(ch) - }() - - return ch -} - // RunResultDetails holds details of the result of commands executed by Run(). type RunResultDetails struct { Node Node diff --git a/pkg/roachprod/install/monitor.go b/pkg/roachprod/install/monitor.go new file mode 100644 index 000000000000..b1dc50efcec8 --- /dev/null +++ b/pkg/roachprod/install/monitor.go @@ -0,0 +1,386 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package install + +import ( + "bufio" + "bytes" + "context" + _ "embed" + "fmt" + "io" + "strconv" + "strings" + "sync" + "text/template" + + "github.com/alessio/shellescape" + "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/errors" +) + +//go:embed scripts/monitor_remote.sh +var monitorRemoteScript string + +//go:embed scripts/monitor_local.sh +var monitorLocalScript string + +// MonitorEvent is an interface for events that can be sent by the +// monitor. +type MonitorEvent interface { + String() string +} + +// MonitorProcessRunning represents the cockroach process running on a +// node. +type MonitorProcessRunning struct { + VirtualClusterName string + SQLInstance int + PID string +} + +// MonitorProcessDead represents the cockroach process dying on a node. +type MonitorProcessDead struct { + VirtualClusterName string + SQLInstance int + ExitCode string +} + +type MonitorError struct { + Err error +} + +// MonitorNoCockroachProcessesError is the error returned when the +// monitor is called on a node that is not running a `cockroach` +// process by the time the monitor runs. +var MonitorNoCockroachProcessesError = errors.New("no cockroach processes running") + +// NodeMonitorInfo is a message describing a cockroach process' status. +type NodeMonitorInfo struct { + // The index of the node (in a SyncedCluster) at which the message originated. + Node Node + // Event describes what happened to the node; it is one of + // MonitorProcessRunning, sent when cockroach is running on a node; + // MonitorProcessDead, when the cockroach process stops running on a + // node; or MonitorError, typically indicate networking issues or + // nodes that have (physically) shut down. + Event MonitorEvent +} + +func (nmi NodeMonitorInfo) String() string { + return fmt.Sprintf("n%d: %s", nmi.Node, nmi.Event.String()) +} + +// MonitorOpts is used to pass the options needed by Monitor. +type MonitorOpts struct { + OneShot bool // Report the status of all targeted nodes once, then exit. +} + +type monitorProcess struct { + processID int + lastProcessID int + status string +} + +type monitorNode struct { + cluster *SyncedCluster + node Node + processes map[string]*monitorProcess + sendEvent func(info NodeMonitorInfo) + opts MonitorOpts +} + +func virtualClusterDesc(name string, instance int) string { + if name == SystemInterfaceName { + return "system interface" + } + return fmt.Sprintf("virtual cluster %q, instance %d", name, instance) +} + +func (e MonitorProcessRunning) String() string { + return fmt.Sprintf("cockroach process for %s is running (PID: %s)", + virtualClusterDesc(e.VirtualClusterName, e.SQLInstance), e.PID, + ) +} + +func (e MonitorProcessDead) String() string { + return fmt.Sprintf("cockroach process for %s died (exit code %s)", + virtualClusterDesc(e.VirtualClusterName, e.SQLInstance), e.ExitCode, + ) +} +func (e MonitorError) String() string { + return fmt.Sprintf("error: %s", e.Err.Error()) +} + +func (m *monitorNode) reset() { + // Reset all process IDs to 0. Each process ID will be updated from the + // monitor script output frame. If a process ID is not updated, it is + // considered dead. + for _, process := range m.processes { + process.lastProcessID = process.processID + process.processID = 0 + } +} + +// checkForProcessChanges compares the process IDs from the current and previous +// monitor script output frames. If a change is detected, it emits an event. +func (m *monitorNode) checkForProcessChanges() { + // Check for dead, or new processes by comparing previous process IDs. + for vcLabel, process := range m.processes { + name, instance, err := VirtualClusterInfoFromLabel(vcLabel) + pid := fmt.Sprintf("%d", process.processID) + if err != nil { + parseError := errors.Wrap(err, "failed to parse virtual cluster label") + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{parseError}}) + continue + } + // Output a dead event whenever the processID changes from a nonzero value + // to any other value. In particular, we emit a dead event when the node + // stops (lastProcessID is nonzero, processID is zero), but not when the + // process then starts again (lastProcessID is zero, processID is nonzero). + if process.lastProcessID != process.processID { + status := process.status + if process.lastProcessID != 0 { + if process.processID != 0 { + status = "unknown" + } + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorProcessDead{ + VirtualClusterName: name, SQLInstance: instance, ExitCode: status, + }}) + } + if process.processID != 0 { + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorProcessRunning{ + VirtualClusterName: name, SQLInstance: instance, PID: pid, + }}) + } + } + } +} + +// processMonitorOutput processes the output of the monitor script. It sets the +// process IDs and statuses of the cockroach processes running on the node, and +// updates the last process IDs for comparison in the next iteration. +func (m *monitorNode) processMonitorOutput(lines []string) { + if len(lines)%2 != 0 { + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{errors.New("bad frame from script")}}) + return + } + + // Reset & update process IDs from the monitor script output frame. + m.reset() + for i := 0; i < len(lines); i += 2 { + vcLine := strings.Split(lines[i], "=") + if len(vcLine) != 2 { + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{errors.New("failed to parse vcs line")}}) + continue + } + processID := 0 + if vcLine[1] != "" { + var parseErr error + processID, parseErr = strconv.Atoi(vcLine[1]) + if parseErr != nil { + parseErr = errors.Wrap(parseErr, "failed to parse process ID") + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{parseErr}}) + continue + } + } + statusLine := strings.Split(lines[i+1], "=") + if len(statusLine) != 2 { + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{errors.New("failed to parse status line")}}) + continue + } + process := m.processes[vcLine[0]] + if process == nil { + process = &monitorProcess{} + m.processes[vcLine[0]] = process + } + process.processID = processID + process.status = statusLine[1] + } +} + +func (m *monitorNode) prepareTemplate() (string, error) { + params := struct { + OneShot bool + RoachprodEnvRegex string + }{ + OneShot: m.opts.OneShot, + RoachprodEnvRegex: m.cluster.roachprodEnvRegex(m.node), + } + monitorScript := monitorRemoteScript + if m.cluster.IsLocal() { + monitorScript = monitorLocalScript + } + t := template.Must(template.New("monitor"). + Funcs(template.FuncMap{"shesc": func(i interface{}) string { + return shellescape.Quote(fmt.Sprint(i)) + }}). + Delims("#{", "#}"). + Parse(monitorScript)) + + var buf bytes.Buffer + if err := t.Execute(&buf, params); err != nil { + return "", errors.Wrap(err, "failed to execute template") + } + return buf.String(), nil +} + +func (m *monitorNode) monitorNode(ctx context.Context, l *logger.Logger) { + script, err := m.prepareTemplate() + if err != nil { + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{err}}) + return + } + + // This is the exception to funneling all SSH traffic through `c.runCmdOnSingleNode` + sess := m.cluster.newSession(l, m.node, script, withDebugDisabled()) + defer sess.Close() + + p, err := sess.StdoutPipe() + if err != nil { + pipeErr := errors.Wrap(err, "failed to read stdout pipe") + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{pipeErr}}) + return + } + // Request a PTY so that the script will receive a SIGPIPE when the + // session is closed. + if err = sess.RequestPty(); err != nil { + ptyErr := errors.Wrap(err, "failed to request PTY") + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{ptyErr}}) + return + } + + var readerWg sync.WaitGroup + readerWg.Add(1) + go func(p io.Reader) { + defer readerWg.Done() + r := bufio.NewReader(p) + lines := make([]string, 0) + for { + lineData, _, err := r.ReadLine() + line := strings.TrimSpace(string(lineData)) + if err == io.EOF { + // Only report EOF errors if we are not in a one-shot mode. + if !m.opts.OneShot { + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{err}}) + } + return + } + if err != nil { + err := errors.Wrap(err, "error reading from session") + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{err}}) + return + } + // An empty line indicates the end of a frame. Process the frame. + if len(line) == 0 { + m.processMonitorOutput(lines) + m.checkForProcessChanges() + lines = make([]string, 0) + continue + } + lines = append(lines, line) + } + }(p) + if err := sess.Start(); err != nil { + err := errors.Wrap(err, "failed to start session") + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{err}}) + return + } + + // Watch for context cancellation, which can happen if the test + // fails, or if the monitor loop exits. + go func() { + <-ctx.Done() + sess.Close() + if pc, ok := p.(io.ReadCloser); ok { + _ = pc.Close() + } + }() + + readerWg.Wait() + // We must call `sess.Wait()` only after finishing reading from the stdout + // pipe. Otherwise, it can be closed under us, causing the reader to loop + // infinitely receiving a non-`io.EOF` error. + if err := sess.Wait(); err != nil { + // If we got an error waiting for the session but the context + // is already canceled, do not send an error through the + // channel; context cancelation happens at the user's request + // or when the test finishes. In either case, the monitor + // should quiesce. Reporting the error is confusing and can be + // noisy in the case of multiple monitors. + if ctx.Err() != nil { + return + } + + err := errors.Wrap(err, "failed to wait for session") + m.sendEvent(NodeMonitorInfo{Node: m.node, Event: MonitorError{err}}) + return + } +} + +// Monitor writes NodeMonitorInfo for the cluster nodes to the returned channel. +// Infos sent to the channel always have the Node the event refers to, and the +// event itself. See documentation for NodeMonitorInfo for possible event types. +// +// If OneShot is true, infos are retrieved only once for each node and the +// channel is subsequently closed; otherwise the process continues indefinitely +// (emitting new information as the status of the cockroach process changes). +// +// Monitor can detect new processes, started after monitor, and will start +// monitoring them, but it cannot detect processes that have already died before +// the monitor starts. +func (c *SyncedCluster) Monitor( + l *logger.Logger, ctx context.Context, opts MonitorOpts, +) chan NodeMonitorInfo { + ch := make(chan NodeMonitorInfo) + nodes := c.TargetNodes() + var wg sync.WaitGroup + monitorCtx, cancel := context.WithCancel(ctx) + + // sendEvent sends the NodeMonitorInfo passed through the channel + // that is listened to by the caller. Bails if the context is + // canceled. + sendEvent := func(info NodeMonitorInfo) { + // if the monitor's context is already canceled, do not attempt to + // send the error down the channel, as it is most likely *caused* + // by the cancelation itself. + if monitorCtx.Err() != nil { + return + } + + select { + case ch <- info: + // We were able to send the info through the channel. + case <-monitorCtx.Done(): + // Don't block trying to send the info. + } + } + + // Start monitoring each node. + wg.Add(len(nodes)) + for _, node := range nodes { + go func(node Node) { + defer wg.Done() + m := &monitorNode{ + cluster: c, + node: node, + sendEvent: sendEvent, + processes: make(map[string]*monitorProcess), + opts: opts, + } + m.monitorNode(monitorCtx, l) + }(node) + } + + // Wait for all monitoring goroutines to finish. + go func() { + wg.Wait() + cancel() + close(ch) + }() + + return ch +} From b957e65c38cf94b0ad53d89a5e47a79434901e00 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Fri, 3 Jan 2025 13:12:44 +0000 Subject: [PATCH 006/126] roachprod: add Monitor data driven tests Previously, Monitor was mostly untested. This change aims to add a few data driven tests using the new mock session functionality. The Monitor logic for checking for process changes have moved to the client, and can now be tested as well. Although we can't test that the shell scripts are correct with these tests, we can test that Monitor interprets the output correctly, and emits the correct events. We can also test that it correctly handles IO errors. The data driven tests specify what the scripts output, and we can then verify the number of events we expect and what those events are. Since events can be out of order, to ensure the tests do not flake we sort the events. Event timing can be tested by chaining multiple output write blocks and event checks. Additionally, a few smalls separate tests are added to check different IO scenarios. Informs: #118214 Epic: None --- pkg/roachprod/install/BUILD.bazel | 2 + pkg/roachprod/install/monitor_test.go | 183 ++++++++++++++++++ .../install/testdata/monitor/bad_frame.txt | 13 ++ .../install/testdata/monitor/dead_process.txt | 30 +++ .../install/testdata/monitor/label_error.txt | 15 ++ .../install/testdata/monitor/multi_node.txt | 35 ++++ .../install/testdata/monitor/new_process.txt | 37 ++++ .../install/testdata/monitor/parse_error.txt | 15 ++ .../testdata/monitor/remote_status.txt | 27 +++ 9 files changed, 357 insertions(+) create mode 100644 pkg/roachprod/install/monitor_test.go create mode 100644 pkg/roachprod/install/testdata/monitor/bad_frame.txt create mode 100644 pkg/roachprod/install/testdata/monitor/dead_process.txt create mode 100644 pkg/roachprod/install/testdata/monitor/label_error.txt create mode 100644 pkg/roachprod/install/testdata/monitor/multi_node.txt create mode 100644 pkg/roachprod/install/testdata/monitor/new_process.txt create mode 100644 pkg/roachprod/install/testdata/monitor/parse_error.txt create mode 100644 pkg/roachprod/install/testdata/monitor/remote_status.txt diff --git a/pkg/roachprod/install/BUILD.bazel b/pkg/roachprod/install/BUILD.bazel index 60c59fda0a2f..18adfbaef620 100644 --- a/pkg/roachprod/install/BUILD.bazel +++ b/pkg/roachprod/install/BUILD.bazel @@ -58,6 +58,7 @@ go_test( srcs = [ "cluster_synced_test.go", "cockroach_test.go", + "monitor_test.go", "services_test.go", "session_test.go", "staging_test.go", @@ -72,6 +73,7 @@ go_test( "//pkg/roachprod/vm/gce/testutils", "//pkg/roachprod/vm/local", "//pkg/testutils/datapathutils", + "//pkg/util/leaktest", "//pkg/util/randutil", "//pkg/util/retry", "@com_github_cockroachdb_datadriven//:datadriven", diff --git a/pkg/roachprod/install/monitor_test.go b/pkg/roachprod/install/monitor_test.go new file mode 100644 index 000000000000..ec0d7937a041 --- /dev/null +++ b/pkg/roachprod/install/monitor_test.go @@ -0,0 +1,183 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package install + +import ( + "context" + "io" + "sort" + "strings" + "sync" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachprod/cloud" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/datadriven" + "github.com/stretchr/testify/require" +) + +func TestMonitorDataDriven(t *testing.T) { + defer leaktest.AfterTest(t)() + datadriven.Walk(t, datapathutils.TestDataPath(t, "monitor"), func(t *testing.T, path string) { + var monitorChan chan NodeMonitorInfo + wg := &sync.WaitGroup{} + sessions := make([]*MockSession, 0) + sessionChan := make([]chan []byte, 0) + ctx := context.Background() + c := &SyncedCluster{ + Cluster: cloud.Cluster{ + Name: "test-cluster", + }, + sessionProvider: func(node Node, cmd string) session { + return sessions[node-1] + }, + } + datadriven.RunTest(t, path, func(t *testing.T, td *datadriven.TestData) string { + switch td.Cmd { + case "monitor": + nodeCount := 1 + td.MaybeScanArgs(t, "nodes", &nodeCount) + c.Nodes = make([]Node, nodeCount) + + wg.Add(nodeCount) + for i := 0; i < nodeCount; i++ { + c.Nodes[i] = Node(i + 1) + sessions = append(sessions, NewMockSession(DefaultMockSessionOptions())) + sessionChan = append(sessionChan, make(chan []byte)) + go func(ch chan []byte, s *MockSession) { + defer wg.Done() + for sc := range ch { + _, err := s.Stdout.Write(sc) + require.NoError(t, err) + } + }(sessionChan[i], sessions[i]) + } + monitorChan = c.Monitor(nilLogger(), ctx, MonitorOpts{}) + return "" + case "write": + node := 1 + td.ScanArgs(t, "node", &node) + input := strings.ReplaceAll(td.Input, `<\n>`, "\n") + sessionChan[node-1] <- []byte(input) + return "" + case "events": + count := 0 + td.ScanArgs(t, "count", &count) + var buf strings.Builder + events := make([]NodeMonitorInfo, 0, count) + for i := 0; i < count; i++ { + e := <-monitorChan + events = append(events, e) + } + // We sort the events to make the test deterministic. To test timing of + // events, multiple write and event command combinations can be used. + sort.Slice(events, func(i, j int) bool { + return strings.Compare(events[i].String(), events[j].String()) < 0 + }) + for _, e := range events { + buf.WriteString(e.String()) + buf.WriteString("\n") + } + return buf.String() + default: + t.Fatalf("unknown command: %s", td.Cmd) + } + return "" + }) + for _, s := range sessions { + s.Close() + } + // We expect EOF errors for all the nodes. This is to ensure that for the + // data driven tests all events have been consumed. Any other error or event + // would be considered a failure. + for e := range monitorChan { + require.IsType(t, MonitorError{}, e.Event) + if e.Event.(MonitorError).Err != io.EOF { + t.Fatalf("unexpected event error: %v", e) + } + } + for _, sc := range sessionChan { + close(sc) + } + wg.Wait() + }) +} + +func TestMonitorCancel(t *testing.T) { + defer leaktest.AfterTest(t)() + c := &SyncedCluster{ + Cluster: cloud.Cluster{ + Name: "test-cluster", + }, + Nodes: []Node{1}, + sessionProvider: func(node Node, cmd string) session { + return NewMockSession(DefaultMockSessionOptions()) + }, + } + + ctx, cancel := context.WithCancel(context.Background()) + monitorChan := c.Monitor(nilLogger(), ctx, MonitorOpts{}) + cancel() + e := <-monitorChan + if e.Event != nil { + t.Fatalf("unexpected event, channel should be closed: %v", e) + } +} + +func TestMonitorEOF(t *testing.T) { + defer leaktest.AfterTest(t)() + s := NewMockSession(DefaultMockSessionOptions()) + c := &SyncedCluster{ + Cluster: cloud.Cluster{ + Name: "test-cluster", + }, + Nodes: []Node{1}, + sessionProvider: func(node Node, cmd string) session { + return s + }, + } + + monitorChan := c.Monitor(nilLogger(), context.Background(), MonitorOpts{}) + s.Close() + e := <-monitorChan + require.IsType(t, MonitorError{}, e.Event) + if e.Event.(MonitorError).Err != io.EOF { + t.Fatalf("unexpected event error: %v", e) + } + e = <-monitorChan + if e.Event != nil { + t.Fatalf("unexpected event, channel should be closed: %v", e) + } +} + +func TestMonitorOneShot(t *testing.T) { + defer leaktest.AfterTest(t)() + s := NewMockSession(DefaultMockSessionOptions()) + c := &SyncedCluster{ + Cluster: cloud.Cluster{ + Name: "test-cluster", + }, + Nodes: []Node{1}, + sessionProvider: func(node Node, cmd string) session { + return s + }, + } + + monitorChan := c.Monitor(nilLogger(), context.Background(), MonitorOpts{OneShot: true}) + go func() { + _, _ = s.Stdout.Write([]byte("system=100\nstatus=unknown\n\n")) + s.Close() + }() + e := <-monitorChan + require.IsType(t, MonitorProcessRunning{}, e.Event) + // We expect EOF errors to be suppressed for one shot mode, and the channel to + // be closed after the first event. + e = <-monitorChan + if e.Event != nil { + t.Fatalf("unexpected event, channel should be closed: %v", e) + } +} diff --git a/pkg/roachprod/install/testdata/monitor/bad_frame.txt b/pkg/roachprod/install/testdata/monitor/bad_frame.txt new file mode 100644 index 000000000000..4ca47aa4ff91 --- /dev/null +++ b/pkg/roachprod/install/testdata/monitor/bad_frame.txt @@ -0,0 +1,13 @@ +monitor nodes=1 +---- + +write node=1 +cockroach-system=500 +cockroach-tenant_0=50 +status=unknown +<\n> +---- + +events count=1 +---- +n1: error: bad frame from script diff --git a/pkg/roachprod/install/testdata/monitor/dead_process.txt b/pkg/roachprod/install/testdata/monitor/dead_process.txt new file mode 100644 index 000000000000..e6748f0a28ae --- /dev/null +++ b/pkg/roachprod/install/testdata/monitor/dead_process.txt @@ -0,0 +1,30 @@ +monitor nodes=1 +---- + +write node=1 +cockroach-system=500 +status=unknown +cockroach-tenant_0=501 +status=unknown +cockroach-tenant2_0=502 +status=unknown +<\n> +---- + +events count=3 +---- +n1: cockroach process for system interface is running (PID: 500) +n1: cockroach process for virtual cluster "tenant", instance 0 is running (PID: 501) +n1: cockroach process for virtual cluster "tenant2", instance 0 is running (PID: 502) + +write node=1 +cockroach-system=500 +status=unknown +cockroach-tenant2_0=502 +status=unknown +<\n> +---- + +events count=1 +---- +n1: cockroach process for virtual cluster "tenant", instance 0 died (exit code unknown) diff --git a/pkg/roachprod/install/testdata/monitor/label_error.txt b/pkg/roachprod/install/testdata/monitor/label_error.txt new file mode 100644 index 000000000000..ef803690ede5 --- /dev/null +++ b/pkg/roachprod/install/testdata/monitor/label_error.txt @@ -0,0 +1,15 @@ +monitor nodes=1 +---- + +write node=1 +cockroach-system=40 +status=unknown +cockroach-tenant0_a=50 +status=unknown +<\n> +---- + +events count=2 +---- +n1: cockroach process for system interface is running (PID: 40) +n1: error: failed to parse virtual cluster label: invalid virtual cluster label: cockroach-tenant0_a diff --git a/pkg/roachprod/install/testdata/monitor/multi_node.txt b/pkg/roachprod/install/testdata/monitor/multi_node.txt new file mode 100644 index 000000000000..32cd622a4515 --- /dev/null +++ b/pkg/roachprod/install/testdata/monitor/multi_node.txt @@ -0,0 +1,35 @@ +monitor nodes=3 +---- + +write node=1 +cockroach-system=500 +status=unknown +cockroach-tenant-a_0=501 +status=unknown +<\n> +---- + +write node=2 +cockroach-system=500 +status=unknown +cockroach-tenant-b_3=501 +status=unknown +<\n> +---- + +write node=3 +cockroach-system=500 +status=unknown +cockroach-tenant-c_7=501 +status=unknown +<\n> +---- + +events count=6 +---- +n1: cockroach process for system interface is running (PID: 500) +n1: cockroach process for virtual cluster "tenant-a", instance 0 is running (PID: 501) +n2: cockroach process for system interface is running (PID: 500) +n2: cockroach process for virtual cluster "tenant-b", instance 3 is running (PID: 501) +n3: cockroach process for system interface is running (PID: 500) +n3: cockroach process for virtual cluster "tenant-c", instance 7 is running (PID: 501) diff --git a/pkg/roachprod/install/testdata/monitor/new_process.txt b/pkg/roachprod/install/testdata/monitor/new_process.txt new file mode 100644 index 000000000000..0417c2aec9c0 --- /dev/null +++ b/pkg/roachprod/install/testdata/monitor/new_process.txt @@ -0,0 +1,37 @@ +monitor nodes=2 +---- + +write node=1 +cockroach-system=500 +status=unknown +<\n> +---- + +events count=1 +---- +n1: cockroach process for system interface is running (PID: 500) + +write node=1 +cockroach-system=500 +status=unknown +cockroach-tenant_0=501 +status=unknown +cockroach-tenant_1=502 +status=unknown +<\n> +---- + +events count=2 +---- +n1: cockroach process for virtual cluster "tenant", instance 0 is running (PID: 501) +n1: cockroach process for virtual cluster "tenant", instance 1 is running (PID: 502) + +write node=2 +cockroach-system=600 +status=unknown +<\n> +---- + +events count=1 +---- +n2: cockroach process for system interface is running (PID: 600) diff --git a/pkg/roachprod/install/testdata/monitor/parse_error.txt b/pkg/roachprod/install/testdata/monitor/parse_error.txt new file mode 100644 index 000000000000..2e047a227dfc --- /dev/null +++ b/pkg/roachprod/install/testdata/monitor/parse_error.txt @@ -0,0 +1,15 @@ +monitor nodes=1 +---- + +write node=1 +cockroach-system=5abc +status=unknown +cockroach-tenant_0=50 +status=unknown +<\n> +---- + +events count=2 +---- +n1: cockroach process for virtual cluster "tenant", instance 0 is running (PID: 50) +n1: error: failed to parse process ID: strconv.Atoi: parsing "5abc": invalid syntax diff --git a/pkg/roachprod/install/testdata/monitor/remote_status.txt b/pkg/roachprod/install/testdata/monitor/remote_status.txt new file mode 100644 index 000000000000..97698d997c2e --- /dev/null +++ b/pkg/roachprod/install/testdata/monitor/remote_status.txt @@ -0,0 +1,27 @@ +monitor nodes=1 +---- + +write node=1 +cockroach-system=500 +status=unknown +cockroach-tenant_0=501 +status=unknown +<\n> +---- + +events count=2 +---- +n1: cockroach process for system interface is running (PID: 500) +n1: cockroach process for virtual cluster "tenant", instance 0 is running (PID: 501) + +write node=1 +cockroach-system=0 +status=7 +cockroach-tenant_0=501 +status=unknown +<\n> +---- + +events count=1 +---- +n1: cockroach process for system interface died (exit code 7) From 23806329ea4708dfc6175876d9d56ee57bed6140 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Fri, 3 Jan 2025 13:12:44 +0000 Subject: [PATCH 007/126] roachtest: remove assert dead nodes Previously, the post assertions would check for dead nodes by using `roachprod.Monitor`. It checks if the data directory of a cockroach node is non-trivial. This is not a reliable way to exclude empty nodes, or to check for dead nodes. It also complicates the Monitor functionality unnecessarily. It also no longer functions, since `roachprod.Monitor` only watches already live processes. This change removes this functionality as it no longer works. An issue (See: #137329) has been created to address this should it still be a requirement. Informs: #118214 Epic: None --- pkg/cmd/roachprod/cli/flags.go | 4 --- pkg/cmd/roachtest/cluster.go | 37 ------------------------- pkg/cmd/roachtest/registry/test_spec.go | 3 ++ pkg/cmd/roachtest/test_runner.go | 12 -------- 4 files changed, 3 insertions(+), 53 deletions(-) diff --git a/pkg/cmd/roachprod/cli/flags.go b/pkg/cmd/roachprod/cli/flags.go index 0fd3883f39c1..ca4b111ea74d 100644 --- a/pkg/cmd/roachprod/cli/flags.go +++ b/pkg/cmd/roachprod/cli/flags.go @@ -298,10 +298,6 @@ func initStageURLCmdFlags(stageURLCmd *cobra.Command) { } func initMonitorCmdFlags(monitorCmd *cobra.Command) { - monitorCmd.Flags().BoolVar(&monitorOpts.IgnoreEmptyNodes, - "ignore-empty-nodes", false, - "Automatically detect the (subset of the given) nodes which to monitor "+ - "based on the presence of a nontrivial data directory.") monitorCmd.Flags().BoolVar(&monitorOpts.OneShot, "oneshot", false, "Report the status of all targeted nodes once, then exit. The exit "+ diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index de1cfbb15ced..e891c17ffd5b 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1480,43 +1480,6 @@ func (c *clusterImpl) FetchVMSpecs(ctx context.Context, l *logger.Logger) error }) } -// assertNoDeadNode returns an error if at least one of the nodes that have a populated -// data dir are found to be not running. It prints both to t.L() and the test -// output. -func (c *clusterImpl) assertNoDeadNode(ctx context.Context, t test.Test) error { - if c.spec.NodeCount == 0 { - // No nodes can happen during unit tests and implies nothing to do. - return nil - } - - t.L().Printf("checking for dead nodes") - eventsCh, err := roachprod.Monitor(ctx, t.L(), c.name, install.MonitorOpts{OneShot: true}) - - // An error here means there was a problem initialising a SyncedCluster. - if err != nil { - return err - } - - deadProcesses := 0 - for info := range eventsCh { - t.L().Printf("%s", info) - - if _, isDeath := info.Event.(install.MonitorProcessDead); isDeath { - deadProcesses++ - } - } - - var plural string - if deadProcesses > 1 { - plural = "es" - } - - if deadProcesses > 0 { - return errors.Newf("%d dead cockroach process%s detected", deadProcesses, plural) - } - return nil -} - func selectedNodesOrDefault( opts []option.Option, defaultNodes option.NodeListOption, ) option.NodeListOption { diff --git a/pkg/cmd/roachtest/registry/test_spec.go b/pkg/cmd/roachtest/registry/test_spec.go index 9387e9512209..9f5fadb13670 100644 --- a/pkg/cmd/roachtest/registry/test_spec.go +++ b/pkg/cmd/roachtest/registry/test_spec.go @@ -196,6 +196,9 @@ const ( // the crdb_internal.invalid_objects virtual table. PostValidationInvalidDescriptors // PostValidationNoDeadNodes checks if there are any dead nodes in the cluster. + // TODO: Deprecate or replace this functionality. + // In its current state it is no longer functional. + // See: https://github.com/cockroachdb/cockroach/issues/137329 for details. PostValidationNoDeadNodes ) diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 73d0ecb6ce36..96e909a59032 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -1480,18 +1480,6 @@ func (r *testRunner) postTestAssertions( postAssertCh := make(chan struct{}) _ = r.stopper.RunAsyncTask(ctx, "test-post-assertions", func(ctx context.Context) { defer close(postAssertCh) - // When a dead node is detected, the subsequent post validation queries are likely - // to hang (reason unclear), and eventually timeout according to the statement_timeout. - // If this occurs frequently enough, we can look at skipping post validations on a node - // failure (or even on any test failure). - if err := c.assertNoDeadNode(ctx, t); err != nil { - // Some tests expect dead nodes, so they may opt out of this check. - if t.spec.SkipPostValidations®istry.PostValidationNoDeadNodes == 0 { - postAssertionErr(err) - } else { - t.L().Printf("dead node(s) detected but expected") - } - } // We collect all the admin health endpoints in parallel, // and select the first one that succeeds to run the validation queries From 0d2b39732491e7c4e445905ce8f38119482a7ea4 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Fri, 3 Jan 2025 13:12:44 +0000 Subject: [PATCH 008/126] roachtest: health status for given nodes Previously, `cluster.HealthStatus` seemed to check all nodes, but rely on the count of a nodes list passed in as a parameter. This change fixes the function to use the passed nodes during the request phase as well. It also adds a small retry mechanism during the 15 seconds it checks the health of the nodes. The issue was found when updating the health check to only look at `c.CRDBNodes()` and exclude the workload node if one is part of the cluster. Informs: #118214 Epic: None --- pkg/cmd/roachtest/cluster.go | 35 +++++++++++++++++++++++--------- pkg/cmd/roachtest/test_runner.go | 2 +- 2 files changed, 26 insertions(+), 11 deletions(-) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index e891c17ffd5b..c385054785f7 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -16,6 +16,7 @@ import ( "log" "math/rand" "net" + "net/http" "net/url" "os" "os/exec" @@ -1513,20 +1514,24 @@ func newHealthStatusResult(node int, status int, body []byte, err error) *Health } } -// HealthStatus returns the result of the /health?ready=1 endpoint for each node. +// HealthStatus returns the result of the /health?ready=1 endpoint for the +// specified nodes. func (c *clusterImpl) HealthStatus( ctx context.Context, l *logger.Logger, nodes option.NodeListOption, ) ([]*HealthStatusResult, error) { - if len(nodes) < 1 { + nodeCount := len(nodes) + if nodeCount < 1 { return nil, nil // unit tests } + adminAddrs, err := c.ExternalAdminUIAddr(ctx, l, nodes) if err != nil { return nil, errors.WithDetail(err, "Unable to get admin UI address(es)") } client := roachtestutil.DefaultHTTPClient(c, l) - getStatus := func(ctx context.Context, node int) *HealthStatusResult { - url := fmt.Sprintf(`https://%s/health?ready=1`, adminAddrs[node-1]) + getStatus := func(ctx context.Context, nodeIndex, node int) *HealthStatusResult { + url := fmt.Sprintf(`https://%s/health?ready=1`, adminAddrs[nodeIndex]) + resp, err := client.Get(ctx, url) if err != nil { return newHealthStatusResult(node, 0, nil, err) @@ -1538,16 +1543,26 @@ func (c *clusterImpl) HealthStatus( return newHealthStatusResult(node, resp.StatusCode, body, err) } - results := make([]*HealthStatusResult, c.spec.NodeCount) + results := make([]*HealthStatusResult, nodeCount) _ = timeutil.RunWithTimeout(ctx, "health status", 15*time.Second, func(ctx context.Context) error { var wg sync.WaitGroup - wg.Add(c.spec.NodeCount) - for i := 1; i <= c.spec.NodeCount; i++ { - go func(node int) { + wg.Add(nodeCount) + for i := 0; i < nodeCount; i++ { + go func() { defer wg.Done() - results[node-1] = getStatus(ctx, node) - }(i) + for { + results[i] = getStatus(ctx, i, nodes[i]) + if results[i].Err == nil && results[i].Status == http.StatusOK { + return + } + select { + case <-ctx.Done(): + return + case <-time.After(3 * time.Second): + } + } + }() } wg.Wait() return nil diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 96e909a59032..badec01858a9 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -1483,7 +1483,7 @@ func (r *testRunner) postTestAssertions( // We collect all the admin health endpoints in parallel, // and select the first one that succeeds to run the validation queries - statuses, err := c.HealthStatus(ctx, t.L(), c.All()) + statuses, err := c.HealthStatus(ctx, t.L(), c.CRDBNodes()) if err != nil { postAssertionErr(errors.WithDetail(err, "Unable to check health status")) } From 164c539470ee5aa30399679168589b18f5c755a5 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Wed, 8 Jan 2025 16:33:30 -0500 Subject: [PATCH 009/126] codeowners: assign export-related files to CDC Release note: None --- .github/CODEOWNERS | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 8fcc1673f3cf..f7177b00ad2a 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -75,6 +75,7 @@ /pkg/sql/delegate/*job*.go @cockroachdb/jobs-prs @cockroachdb/disaster-recovery /pkg/sql/importer/ @cockroachdb/sql-queries-prs +/pkg/sql/importer/export* @cockroachdb/cdc-prs /pkg/ccl/importerccl/ @cockroachdb/sql-queries-prs /pkg/sql/appstatspb @cockroachdb/obs-prs From 633a55f533a47f800fa6fabd5f5871a3d78a836a Mon Sep 17 00:00:00 2001 From: Ibrahim Kettaneh Date: Fri, 3 Jan 2025 13:36:17 -0500 Subject: [PATCH 010/126] raft: add and compute lastComputedLeadSupportUntil This commit adds a new field to the fortificationTracker called lastComputedLeadSupportUntil. This field is updated on every tick, and on every config change / fortification response. Future commits will read this field directly without recomputing the LeadSupportUntil. References: #137264 Release note: None --- pkg/raft/raft.go | 7 ++++ pkg/raft/tracker/fortificationtracker.go | 46 +++++++++++++++++++++++- 2 files changed, 52 insertions(+), 1 deletion(-) diff --git a/pkg/raft/raft.go b/pkg/raft/raft.go index 11c08d48a47a..c09a6d699b22 100644 --- a/pkg/raft/raft.go +++ b/pkg/raft/raft.go @@ -1839,6 +1839,9 @@ func (r *raft) logMsgHigherTerm(m pb.Message, suffix redact.SafeString) { type stepFunc func(r *raft, m pb.Message) error func stepLeader(r *raft, m pb.Message) error { + // Compute the LeadSupportUntil on every tick. + r.fortificationTracker.UpdateComputedLeadSupportUntil(r.state) + // These message types do not require any progress for m.From. switch m.Type { case pb.MsgBeat: @@ -2755,6 +2758,10 @@ func (r *raft) switchToConfig(cfg quorum.Config, progressMap tracker.ProgressMap r.logger.Panicf("%x leader removed from configuration %s", r.id, r.config) } + // Config changes might cause the LeadSupportUntil to change, we need to + // recalculate it here. + r.fortificationTracker.UpdateComputedLeadSupportUntil(r.state) + if r.isLearner { // This node is leader and was demoted, step down. // diff --git a/pkg/raft/tracker/fortificationtracker.go b/pkg/raft/tracker/fortificationtracker.go index 12fd0d5a675b..77899a56487a 100644 --- a/pkg/raft/tracker/fortificationtracker.go +++ b/pkg/raft/tracker/fortificationtracker.go @@ -92,6 +92,14 @@ type FortificationTracker struct { // to campaign at a lower term, then learns about the higher term, and then // use it in the next campaign attempt. steppingDownTerm uint64 + + // computedLeadSupportUntil is the last computed LeadSupportUntil. We + // update this value on (1) Every tick, (2) Every time a new fortification is + // recorded, and (3) On config changes. Callers of LeadSupportUntil will get + // this cached version of LeadSupportUntil, which is good because + // LeadSupportUntil is called by every request trying to evaluate the lease's + // status. + computedLeadSupportUntil hlc.Timestamp } // NewFortificationTracker initializes a FortificationTracker. @@ -128,6 +136,9 @@ func (ft *FortificationTracker) RecordFortification(id pb.PeerID, epoch pb.Epoch // The supported epoch should never regress. Guard against out of order // delivery of fortify responses by using max. ft.fortification[id] = max(ft.fortification[id], epoch) + // Every time a new follower has fortified us, we need to recompute the + // LeadSupportUntil since it might have changed. + ft.UpdateComputedLeadSupportUntil(pb.StateLeader) } // Reset clears out any previously tracked fortification and prepares the @@ -143,6 +154,7 @@ func (ft *FortificationTracker) Reset(term uint64) { ft.leaderMaxSupported.Reset() ft.steppingDown = false ft.steppingDownTerm = 0 + ft.computedLeadSupportUntil = hlc.Timestamp{} } // IsFortifiedBy returns whether the follower fortifies the leader or not. @@ -188,6 +200,38 @@ func (ft *FortificationTracker) LeadSupportUntil(state pb.StateType) hlc.Timesta return ft.leaderMaxSupported.Forward(leadSupportUntil) } +// UpdateComputedLeadSupportUntil updates the field +// computedLeadSupportUntil by computing the LeadSupportExpiration. +func (ft *FortificationTracker) UpdateComputedLeadSupportUntil(state pb.StateType) hlc.Timestamp { + if state != pb.StateLeader { + panic("UpdateComputedLeadSupportUntil should only be called by the leader") + } + + if len(ft.fortification) == 0 { + ft.computedLeadSupportUntil = hlc.Timestamp{} + return ft.computedLeadSupportUntil // fast-path for no fortification + } + + // TODO(ibrahim): avoid this map allocation as we're calling LeadSupportUntil + // on every tick, on every new fortification, and on config changes. + supportExpMap := make(map[pb.PeerID]hlc.Timestamp) + ft.config.Voters.Visit(func(id pb.PeerID) { + if supportEpoch, ok := ft.fortification[id]; ok { + curEpoch, curExp := ft.storeLiveness.SupportFrom(id) + // NB: We can't assert that supportEpoch <= curEpoch because there may be + // a race between a successful MsgFortifyLeaderResp and the store liveness + // heartbeat response that lets the leader know the follower's store is + // supporting the leader's store at the epoch in the MsgFortifyLeaderResp + // message. + if curEpoch == supportEpoch { + supportExpMap[id] = curExp + } + } + }) + ft.computedLeadSupportUntil = ft.config.Voters.LeadSupportExpiration(supportExpMap) + return ft.computedLeadSupportUntil +} + // computeLeadSupportUntil computes the timestamp until which the leader is // guaranteed fortification using the current quorum configuration. // @@ -202,7 +246,7 @@ func (ft *FortificationTracker) computeLeadSupportUntil(state pb.StateType) hlc. return hlc.Timestamp{} // fast-path for no fortification } - // TODO(arul): avoid this map allocation as we're calling LeadSupportUntil + // TODO(ibrahim): avoid this map allocation as we're calling LeadSupportUntil // from hot paths. supportExpMap := make(map[pb.PeerID]hlc.Timestamp) ft.config.Voters.Visit(func(id pb.PeerID) { From d056d59a1b3ed31031c50da6d58dd1117d62e1e7 Mon Sep 17 00:00:00 2001 From: Ibrahim Kettaneh Date: Fri, 3 Jan 2025 13:47:52 -0500 Subject: [PATCH 011/126] raft: read the cached value lastComputedLeadSupportUntil This commit replaces the code that used to calculate the LeadSupportUntil with a read to the cached value lastComputedLeadSupportUntil. This should make it very cheap to get the LeadSupportUntil. References: #137264 Release note: None --- pkg/raft/raft.go | 4 +- pkg/raft/tracker/fortificationtracker.go | 50 +++---------------- pkg/raft/tracker/fortificationtracker_test.go | 1 + 3 files changed, 11 insertions(+), 44 deletions(-) diff --git a/pkg/raft/raft.go b/pkg/raft/raft.go index c09a6d699b22..f1bcdf60b1c2 100644 --- a/pkg/raft/raft.go +++ b/pkg/raft/raft.go @@ -1840,7 +1840,7 @@ type stepFunc func(r *raft, m pb.Message) error func stepLeader(r *raft, m pb.Message) error { // Compute the LeadSupportUntil on every tick. - r.fortificationTracker.UpdateComputedLeadSupportUntil(r.state) + r.fortificationTracker.ComputeLeadSupportUntil(r.state) // These message types do not require any progress for m.From. switch m.Type { @@ -2760,7 +2760,7 @@ func (r *raft) switchToConfig(cfg quorum.Config, progressMap tracker.ProgressMap // Config changes might cause the LeadSupportUntil to change, we need to // recalculate it here. - r.fortificationTracker.UpdateComputedLeadSupportUntil(r.state) + r.fortificationTracker.ComputeLeadSupportUntil(r.state) if r.isLearner { // This node is leader and was demoted, step down. diff --git a/pkg/raft/tracker/fortificationtracker.go b/pkg/raft/tracker/fortificationtracker.go index 77899a56487a..80aba5e6e637 100644 --- a/pkg/raft/tracker/fortificationtracker.go +++ b/pkg/raft/tracker/fortificationtracker.go @@ -138,7 +138,7 @@ func (ft *FortificationTracker) RecordFortification(id pb.PeerID, epoch pb.Epoch ft.fortification[id] = max(ft.fortification[id], epoch) // Every time a new follower has fortified us, we need to recompute the // LeadSupportUntil since it might have changed. - ft.UpdateComputedLeadSupportUntil(pb.StateLeader) + ft.ComputeLeadSupportUntil(pb.StateLeader) } // Reset clears out any previously tracked fortification and prepares the @@ -194,17 +194,16 @@ func (ft *FortificationTracker) LeadSupportUntil(state pb.StateType) hlc.Timesta return ft.leaderMaxSupported.Load() } - // Compute the lead support using the current configuration and forward the - // leaderMaxSupported to avoid regressions when the configuration changes. - leadSupportUntil := ft.computeLeadSupportUntil(state) - return ft.leaderMaxSupported.Forward(leadSupportUntil) + // Forward the leaderMaxSupported to avoid regressions when the configuration + // changes. + return ft.leaderMaxSupported.Forward(ft.computedLeadSupportUntil) } -// UpdateComputedLeadSupportUntil updates the field +// ComputeLeadSupportUntil updates the field // computedLeadSupportUntil by computing the LeadSupportExpiration. -func (ft *FortificationTracker) UpdateComputedLeadSupportUntil(state pb.StateType) hlc.Timestamp { +func (ft *FortificationTracker) ComputeLeadSupportUntil(state pb.StateType) hlc.Timestamp { if state != pb.StateLeader { - panic("UpdateComputedLeadSupportUntil should only be called by the leader") + panic("ComputeLeadSupportUntil should only be called by the leader") } if len(ft.fortification) == 0 { @@ -232,39 +231,6 @@ func (ft *FortificationTracker) UpdateComputedLeadSupportUntil(state pb.StateTyp return ft.computedLeadSupportUntil } -// computeLeadSupportUntil computes the timestamp until which the leader is -// guaranteed fortification using the current quorum configuration. -// -// Unlike LeadSupportUntil, this computation does not provide a guarantee of -// monotonicity. Specifically, its result may regress after a configuration -// change. -func (ft *FortificationTracker) computeLeadSupportUntil(state pb.StateType) hlc.Timestamp { - if state != pb.StateLeader { - panic("computeLeadSupportUntil should only be called by the leader") - } - if len(ft.fortification) == 0 { - return hlc.Timestamp{} // fast-path for no fortification - } - - // TODO(ibrahim): avoid this map allocation as we're calling LeadSupportUntil - // from hot paths. - supportExpMap := make(map[pb.PeerID]hlc.Timestamp) - ft.config.Voters.Visit(func(id pb.PeerID) { - if supportEpoch, ok := ft.fortification[id]; ok { - curEpoch, curExp := ft.storeLiveness.SupportFrom(id) - // NB: We can't assert that supportEpoch <= curEpoch because there may be - // a race between a successful MsgFortifyLeaderResp and the store liveness - // heartbeat response that lets the leader know the follower's store is - // supporting the leader's store at the epoch in the MsgFortifyLeaderResp - // message. - if curEpoch == supportEpoch { - supportExpMap[id] = curExp - } - } - }) - return ft.config.Voters.LeadSupportExpiration(supportExpMap) -} - // CanDefortify returns whether the caller can safely[1] de-fortify the term // based on the state tracked by the FortificationTracker. // @@ -399,7 +365,7 @@ func (ft *FortificationTracker) ConfigChangeSafe() bool { // previous configuration, which is reflected in leaderMaxSupported. // // NB: Only run by the leader. - return ft.leaderMaxSupported.Load().LessEq(ft.computeLeadSupportUntil(pb.StateLeader)) + return ft.leaderMaxSupported.Load().LessEq(ft.computedLeadSupportUntil) } // QuorumActive returns whether the leader is currently supported by a quorum or diff --git a/pkg/raft/tracker/fortificationtracker_test.go b/pkg/raft/tracker/fortificationtracker_test.go index 3c2fefd738ab..4c8094542aec 100644 --- a/pkg/raft/tracker/fortificationtracker_test.go +++ b/pkg/raft/tracker/fortificationtracker_test.go @@ -899,6 +899,7 @@ func TestConfigChangeSafe(t *testing.T) { tc.afterConfigChange(&mockLiveness, ft) + ft.ComputeLeadSupportUntil(pb.StateLeader) require.Equal(t, tc.expConfigChangeSafe, ft.ConfigChangeSafe()) require.Equal(t, tc.expLeadSupportUntil, ft.LeadSupportUntil(pb.StateLeader)) } From 321cdd8ac389eb19d271f6b3c28581e0511d00c8 Mon Sep 17 00:00:00 2001 From: Ibrahim Kettaneh Date: Fri, 3 Jan 2025 14:28:20 -0500 Subject: [PATCH 012/126] raft: remove LeadSupportStatus and add it to BasicStatus This commit removes the Raft status LeadSupportStatus, and adds it to BasicStatus. This is now possible because getting LeadSupportUntil is cheap due to caching References: #137264 Release note: None --- pkg/kv/kvserver/leases/status.go | 2 +- pkg/kv/kvserver/leases/status_test.go | 6 +-- pkg/kv/kvserver/replica.go | 7 --- pkg/kv/kvserver/replica_metrics.go | 4 +- pkg/kv/kvserver/replica_range_lease.go | 8 +-- pkg/kv/kvserver/replica_test.go | 70 ++++++++++++-------------- pkg/raft/raft_test.go | 4 +- pkg/raft/rawnode.go | 6 --- pkg/raft/status.go | 35 ++++--------- 9 files changed, 53 insertions(+), 89 deletions(-) diff --git a/pkg/kv/kvserver/leases/status.go b/pkg/kv/kvserver/leases/status.go index 69b340e3cb56..6f17b9b7eb7c 100644 --- a/pkg/kv/kvserver/leases/status.go +++ b/pkg/kv/kvserver/leases/status.go @@ -27,7 +27,7 @@ type StatusInput struct { MinValidObservedTs hlc.ClockTimestamp // Information about raft. - RaftStatus raft.LeadSupportStatus + RaftStatus raft.BasicStatus // The current time and the time of the request to evaluate the lease for. Now hlc.ClockTimestamp diff --git a/pkg/kv/kvserver/leases/status_test.go b/pkg/kv/kvserver/leases/status_test.go index 41f2be2532d9..7c17cfcc11f9 100644 --- a/pkg/kv/kvserver/leases/status_test.go +++ b/pkg/kv/kvserver/leases/status_test.go @@ -69,8 +69,8 @@ func TestStatus(t *testing.T) { leaderLeaseRemote := leaderLease leaderLeaseRemote.Replica = repl2 - raftStatus := func(state raftpb.StateType, term uint64, leadSupport hlc.Timestamp) raft.LeadSupportStatus { - var s raft.LeadSupportStatus + raftStatus := func(state raftpb.StateType, term uint64, leadSupport hlc.Timestamp) raft.BasicStatus { + var s raft.BasicStatus s.ID = raftpb.PeerID(repl1.ReplicaID) s.RaftState = state s.Term = term @@ -98,7 +98,7 @@ func TestStatus(t *testing.T) { now hlc.ClockTimestamp minProposedTS hlc.ClockTimestamp reqTS hlc.Timestamp - raftStatus raft.LeadSupportStatus + raftStatus raft.BasicStatus liveness livenesspb.Liveness want kvserverpb.LeaseState wantErr string diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 01e7b11490af..c4c8ea517195 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1686,13 +1686,6 @@ func (r *Replica) raftBasicStatusRLocked() raft.BasicStatus { return raft.BasicStatus{} } -func (r *Replica) raftLeadSupportStatusRLocked() raft.LeadSupportStatus { - if rg := r.mu.internalRaftGroup; rg != nil { - return rg.LeadSupportStatus() - } - return raft.LeadSupportStatus{} -} - // RACv2Status returns the status of the RACv2 range controller of this replica. // Returns an empty struct if there is no RACv2 range controller, i.e. this // replica is not the leader or is not running RACv2. diff --git a/pkg/kv/kvserver/replica_metrics.go b/pkg/kv/kvserver/replica_metrics.go index 37eb650401a3..bcf92d839e39 100644 --- a/pkg/kv/kvserver/replica_metrics.go +++ b/pkg/kv/kvserver/replica_metrics.go @@ -101,7 +101,6 @@ func (r *Replica) Metrics( clusterNodes: clusterNodes, desc: r.shMu.state.Desc, raftStatus: r.raftSparseStatusRLocked(), - leadSupportStatus: r.raftLeadSupportStatusRLocked(), now: now, leaseStatus: r.leaseStatusAtRLocked(ctx, now), storeID: r.store.StoreID(), @@ -134,7 +133,6 @@ type calcReplicaMetricsInput struct { clusterNodes int desc *roachpb.RangeDescriptor raftStatus *raft.SparseStatus - leadSupportStatus raft.LeadSupportStatus now hlc.ClockTimestamp leaseStatus kvserverpb.LeaseStatus storeID roachpb.StoreID @@ -190,7 +188,7 @@ func calcReplicaMetrics(d calcReplicaMetricsInput) ReplicaMetrics { if leader { leaderBehindCount = calcBehindCount(d.raftStatus, d.desc, d.vitalityMap) leaderPausedFollowerCount = int64(len(d.paused)) - leaderNotFortified = d.leadSupportStatus.LeadSupportUntil.Less(d.now.ToTimestamp()) + leaderNotFortified = d.raftStatus.LeadSupportUntil.Less(d.now.ToTimestamp()) } return ReplicaMetrics{ diff --git a/pkg/kv/kvserver/replica_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index f69221285229..c0f9703a8dbf 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -758,13 +758,9 @@ func (r *Replica) leaseStatusForRequestRLocked( RequestTs: reqTS, Lease: *r.shMu.state.Lease, } - // TODO(nvanbenschoten): evaluate whether this is too expensive to compute for - // every lease status request. We may need to cache this result. If, at that - // time, we determine that it is sufficiently cheap, we should either compute - // it unconditionally, regardless of the lease type or let leases.Status - // decide when to compute it. See #125255. + if in.Lease.Type() == roachpb.LeaseLeader { - in.RaftStatus = r.raftLeadSupportStatusRLocked() + in.RaftStatus = r.raftBasicStatusRLocked() } return leases.Status(ctx, r.store.cfg.NodeLiveness, in) } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 45f9f6222bf3..64818e2d7927 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -9016,7 +9016,8 @@ func TestReplicaMetrics(t *testing.T) { return m } - status := func(lead raftpb.PeerID, progress map[raftpb.PeerID]tracker.Progress) *raft.SparseStatus { + status := func(lead raftpb.PeerID, progress map[raftpb.PeerID]tracker.Progress, + leadSupportUntil hlc.Timestamp) *raft.SparseStatus { status := &raft.SparseStatus{ Progress: progress, } @@ -9029,10 +9030,7 @@ func TestReplicaMetrics(t *testing.T) { status.SoftState.RaftState = raftpb.StateFollower } status.HardState.Lead = lead - return status - } - leadSupportedStatus := func(leadSupportedUntil hlc.Timestamp) raft.LeadSupportStatus { - status := raft.LeadSupportStatus{LeadSupportUntil: leadSupportedUntil} + status.BasicStatus.LeadSupportUntil = leadSupportUntil return status } desc := func(ids ...int) roachpb.RangeDescriptor { @@ -9058,17 +9056,16 @@ func TestReplicaMetrics(t *testing.T) { tc.StartWithStoreConfig(ctx, t, stopper, cfg) testCases := []struct { - replicas int32 - storeID roachpb.StoreID - desc roachpb.RangeDescriptor - raftStatus *raft.SparseStatus - liveness livenesspb.NodeVitalityInterface - raftLogSize int64 - leaderSupportStatus raft.LeadSupportStatus - expected ReplicaMetrics + replicas int32 + storeID roachpb.StoreID + desc roachpb.RangeDescriptor + raftStatus *raft.SparseStatus + liveness livenesspb.NodeVitalityInterface + raftLogSize int64 + expected ReplicaMetrics }{ // The leader of a 1-replica range is up. - {1, 1, desc(1), status(1, progress(2)), live(1), 0, leadSupportedStatus(hlc.Timestamp{}), + {1, 1, desc(1), status(1, progress(2), hlc.Timestamp{}), live(1), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9079,7 +9076,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{1, 0, 0}, }}, // The leader of a 2-replica range is up (only 1 replica present). - {2, 1, desc(1), status(1, progress(2)), live(1), 0, leadSupportedStatus(hlc.Timestamp{}), + {2, 1, desc(1), status(1, progress(2), hlc.Timestamp{}), live(1), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9090,7 +9087,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{1, 0, 0}, }}, // The leader of a 2-replica range is up. - {2, 1, desc(1, 2), status(1, progress(2)), live(1), 0, leadSupportedStatus(hlc.Timestamp{}), + {2, 1, desc(1, 2), status(1, progress(2), hlc.Timestamp{}), live(1), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9101,7 +9098,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{1, 0, 0}, }}, // Both replicas of a 2-replica range are up to date. - {2, 1, desc(1, 2), status(1, progress(2, 2)), live(1, 2), 0, leadSupportedStatus(hlc.Timestamp{}), + {2, 1, desc(1, 2), status(1, progress(2, 2), hlc.Timestamp{}), live(1, 2), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9112,7 +9109,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{2, 0, 0}, }}, // Both replicas of a 2-replica range are up to date (local replica is not leader) - {2, 2, desc(1, 2), status(2, progress(2, 2)), live(1, 2), 0, leadSupportedStatus(hlc.Timestamp{}), + {2, 2, desc(1, 2), status(2, progress(2, 2), hlc.Timestamp{}), live(1, 2), 0, ReplicaMetrics{ Leader: false, RangeCounter: false, @@ -9121,7 +9118,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{0, 0, 0}, }}, // Both replicas of a 2-replica range are live, but follower is behind. - {2, 1, desc(1, 2), status(1, progress(2, 1)), live(1, 2), 0, leadSupportedStatus(hlc.Timestamp{}), + {2, 1, desc(1, 2), status(1, progress(2, 1), hlc.Timestamp{}), live(1, 2), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9132,7 +9129,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{2, 0, 0}, }}, // Both replicas of a 2-replica range are up to date, but follower is dead. - {2, 1, desc(1, 2), status(1, progress(2, 2)), live(1), 0, leadSupportedStatus(hlc.Timestamp{}), + {2, 1, desc(1, 2), status(1, progress(2, 2), hlc.Timestamp{}), live(1), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9143,7 +9140,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{2, 0, 0}, }}, // The leader of a 3-replica range is up. - {3, 1, desc(1, 2, 3), status(1, progress(1)), live(1), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 1, desc(1, 2, 3), status(1, progress(1), hlc.Timestamp{}), live(1), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9154,7 +9151,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{1, 0, 0}, }}, // All replicas of a 3-replica range are up to date. - {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 2)), live(1, 2, 3), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 2), hlc.Timestamp{}), live(1, 2, 3), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9166,7 +9163,7 @@ func TestReplicaMetrics(t *testing.T) { }}, // All replicas of a 3-replica range are up to date (match = 0 is // considered up to date). - {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 0)), live(1, 2, 3), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 0), hlc.Timestamp{}), live(1, 2, 3), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9177,7 +9174,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{3, 0, 0}, }}, // All replicas of a 3-replica range are live but one replica is behind. - {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 1)), live(1, 2, 3), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 1), hlc.Timestamp{}), live(1, 2, 3), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9188,7 +9185,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{3, 0, 0}, }}, // All replicas of a 3-replica range are live but two replicas are behind. - {3, 1, desc(1, 2, 3), status(1, progress(2, 1, 1)), live(1, 2, 3), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 1, desc(1, 2, 3), status(1, progress(2, 1, 1), hlc.Timestamp{}), live(1, 2, 3), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9199,7 +9196,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{3, 0, 0}, }}, // All replicas of a 3-replica range are up to date, but one replica is dead. - {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 2)), live(1, 2), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 2), hlc.Timestamp{}), live(1, 2), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9210,7 +9207,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{3, 0, 0}, }}, // All replicas of a 3-replica range are up to date, but two replicas are dead. - {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 2)), live(1), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 1, desc(1, 2, 3), status(1, progress(2, 2, 2), hlc.Timestamp{}), live(1), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9222,7 +9219,7 @@ func TestReplicaMetrics(t *testing.T) { }}, // All replicas of a 3-replica range are up to date, but two replicas are // dead, including the leader. - {3, 2, desc(1, 2, 3), status(0, progress(2, 2, 2)), live(2), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 2, desc(1, 2, 3), status(0, progress(2, 2, 2), hlc.Timestamp{}), live(2), 0, ReplicaMetrics{ Leader: false, RangeCounter: true, @@ -9232,7 +9229,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{0, 0, 0}, }}, // Range has no leader, local replica is the range counter. - {3, 1, desc(1, 2, 3), status(0, progress(2, 2, 2)), live(1, 2, 3), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 1, desc(1, 2, 3), status(0, progress(2, 2, 2), hlc.Timestamp{}), live(1, 2, 3), 0, ReplicaMetrics{ Leader: false, RangeCounter: true, @@ -9241,7 +9238,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{0, 0, 0}, }}, // Range has no leader, local replica is the range counter. - {3, 3, desc(3, 2, 1), status(0, progress(2, 2, 2)), live(1, 2, 3), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 3, desc(3, 2, 1), status(0, progress(2, 2, 2), hlc.Timestamp{}), live(1, 2, 3), 0, ReplicaMetrics{ Leader: false, RangeCounter: true, @@ -9250,7 +9247,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{0, 0, 0}, }}, // Range has no leader, local replica is not the range counter. - {3, 2, desc(1, 2, 3), status(0, progress(2, 2, 2)), live(1, 2, 3), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 2, desc(1, 2, 3), status(0, progress(2, 2, 2), hlc.Timestamp{}), live(1, 2, 3), 0, ReplicaMetrics{ Leader: false, RangeCounter: false, @@ -9259,7 +9256,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{0, 0, 0}, }}, // Range has no leader, local replica is not the range counter. - {3, 3, desc(1, 2, 3), status(0, progress(2, 2, 2)), live(1, 2, 3), 0, leadSupportedStatus(hlc.Timestamp{}), + {3, 3, desc(1, 2, 3), status(0, progress(2, 2, 2), hlc.Timestamp{}), live(1, 2, 3), 0, ReplicaMetrics{ Leader: false, RangeCounter: false, @@ -9268,7 +9265,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{0, 0, 0}, }}, // The leader of a 1-replica range is up and raft log is too large. - {1, 1, desc(1), status(1, progress(2)), live(1), 5 * cfg.RaftLogTruncationThreshold, leadSupportedStatus(hlc.Timestamp{}), + {1, 1, desc(1), status(1, progress(2), hlc.Timestamp{}), live(1), 5 * cfg.RaftLogTruncationThreshold, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9280,7 +9277,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{1, 0, 0}, }}, // The leader of a 1-replica range is up, and the leader support expired. - {1, 1, desc(1), status(1, progress(2)), live(1), 0, leadSupportedStatus(hlc.MinTimestamp), + {1, 1, desc(1), status(1, progress(2), hlc.MinTimestamp), live(1), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9291,7 +9288,7 @@ func TestReplicaMetrics(t *testing.T) { RaftFlowStateCounts: [3]int64{1, 0, 0}, }}, // The leader of a 1-replica range is up, and the support hasn't expired. - {1, 1, desc(1), status(1, progress(2)), live(1), 0, leadSupportedStatus(hlc.MaxTimestamp), + {1, 1, desc(1), status(1, progress(2), hlc.MaxTimestamp), live(1), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9304,7 +9301,7 @@ func TestReplicaMetrics(t *testing.T) { // 2 replicas are in StateReplicate, and one in StateSnapshot. {3, 1, desc(1, 2, 3), status(1, withStates(progress(2, 1, 1), tracker.StateReplicate, tracker.StateReplicate, tracker.StateSnapshot, - )), live(1, 2, 3), 0, leadSupportedStatus(hlc.Timestamp{}), + ), hlc.Timestamp{}), live(1, 2, 3), 0, ReplicaMetrics{ Leader: true, RangeCounter: true, @@ -9337,7 +9334,6 @@ func TestReplicaMetrics(t *testing.T) { raftLogSize: c.raftLogSize, raftLogSizeTrusted: true, now: tc.Clock().NowAsClockTimestamp(), - leadSupportStatus: c.leaderSupportStatus, }) require.Equal(t, c.expected, metrics) }) diff --git a/pkg/raft/raft_test.go b/pkg/raft/raft_test.go index 58deb7905d0d..62022f06dd5a 100644 --- a/pkg/raft/raft_test.go +++ b/pkg/raft/raft_test.go @@ -2177,7 +2177,7 @@ func testFreeStuckCandidateWithCheckQuorum(t *testing.T, storeLivenessEnabled bo nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) assert.Equal(t, pb.StateLeader, a.state) if storeLivenessEnabled { - assert.Equal(t, hlc.MaxTimestamp, getLeadSupportStatus(a).LeadSupportUntil) + assert.Equal(t, hlc.MaxTimestamp, getBasicStatus(a).LeadSupportUntil) } nt.isolate(1) @@ -4533,7 +4533,7 @@ func testPreVoteMigrationWithFreeStuckPreCandidate(t *testing.T, storeLivenessEn assert.Equal(t, pb.StateLeader, n1.state) if storeLivenessEnabled { - assert.Equal(t, hlc.MaxTimestamp, getLeadSupportStatus(n1).LeadSupportUntil) + assert.Equal(t, hlc.MaxTimestamp, getBasicStatus(n1).LeadSupportUntil) } if storeLivenessEnabled { diff --git a/pkg/raft/rawnode.go b/pkg/raft/rawnode.go index 6895d80bc99c..7f0cd947435f 100644 --- a/pkg/raft/rawnode.go +++ b/pkg/raft/rawnode.go @@ -617,12 +617,6 @@ func (rn *RawNode) SparseStatus() SparseStatus { return getSparseStatus(rn.raft) } -// LeadSupportStatus returns a LeadSupportStatus. Notably, it only includes -// leader support information. -func (rn *RawNode) LeadSupportStatus() LeadSupportStatus { - return getLeadSupportStatus(rn.raft) -} - // ProgressType indicates the type of replica a Progress corresponds to. type ProgressType byte diff --git a/pkg/raft/status.go b/pkg/raft/status.go index 14c80ca4a2be..4dbe9cb2a645 100644 --- a/pkg/raft/status.go +++ b/pkg/raft/status.go @@ -31,9 +31,8 @@ import ( // The Progress is only populated on the leader. type Status struct { BasicStatus - Config quorum.Config - Progress map[pb.PeerID]tracker.Progress - LeadSupportUntil hlc.Timestamp + Config quorum.Config + Progress map[pb.PeerID]tracker.Progress } // SparseStatus is a variant of Status without Config or Progress.Inflights, @@ -43,13 +42,6 @@ type SparseStatus struct { Progress map[pb.PeerID]tracker.Progress } -// LeadSupportStatus is a variant of Status without Config or Progress, which -// are expensive to copy. -type LeadSupportStatus struct { - BasicStatus - LeadSupportUntil hlc.Timestamp -} - // BasicStatus contains basic information about the Raft peer. It does not allocate. type BasicStatus struct { ID pb.PeerID @@ -59,7 +51,8 @@ type BasicStatus struct { Applied uint64 - LeadTransferee pb.PeerID + LeadTransferee pb.PeerID + LeadSupportUntil hlc.Timestamp } // Empty returns true if the receiver is empty. @@ -101,6 +94,13 @@ func getBasicStatus(r *raft) BasicStatus { s.HardState = r.hardState() s.SoftState = r.softState() s.Applied = r.raftLog.applied + + // NOTE: we assign to LeadSupportUntil even if RaftState is not currently + // StateLeader. The replica may have been the leader and stepped down to a + // follower before its lead support ran out. + //s.LeadSupportUntil = hlc.Timestamp{} + s.LeadSupportUntil = r.fortificationTracker.LeadSupportUntil(r.state) + assertTrue((s.RaftState == pb.StateLeader) == (s.Lead == r.id), "inconsistent lead / raft state") return s } @@ -113,10 +113,6 @@ func getStatus(r *raft) Status { s.Progress = getProgressCopy(r) } s.Config = r.config.Clone() - // NOTE: we assign to LeadSupportUntil even if RaftState is not currently - // StateLeader. The replica may have been the leader and stepped down to a - // follower before its lead support ran out. - s.LeadSupportUntil = r.fortificationTracker.LeadSupportUntil(r.state) return s } @@ -135,15 +131,6 @@ func getSparseStatus(r *raft) SparseStatus { return s } -// getLeadSupportStatus gets a copy of the current raft status with only the -// leader support information included. -func getLeadSupportStatus(r *raft) LeadSupportStatus { - var s LeadSupportStatus - s.BasicStatus = getBasicStatus(r) - s.LeadSupportUntil = r.fortificationTracker.LeadSupportUntil(r.state) - return s -} - // MarshalJSON translates the raft status into JSON. func (s Status) MarshalJSON() ([]byte, error) { j := fmt.Sprintf(`{"id":"%x","term":%d,"vote":"%x","commit":%d,"lead":"%x","leadEpoch":"%d","raftState":%q,"applied":%d,"progress":{`, From 7b3ded52615978a3b7c91c2dd1f66ead0425fd32 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 9 Jan 2025 10:49:18 -0500 Subject: [PATCH 013/126] opt/bench: fix benchmark regression PR #138641 caused extra allocations for plan gist factories in optimizer benchmarks. These allocations should not be included in benchmark results, so they have been eliminated. Release note: None --- pkg/sql/opt/bench/bench_test.go | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/pkg/sql/opt/bench/bench_test.go b/pkg/sql/opt/bench/bench_test.go index 7386ee67bd8a..be3417ae0ba0 100644 --- a/pkg/sql/opt/bench/bench_test.go +++ b/pkg/sql/opt/bench/bench_test.go @@ -813,6 +813,7 @@ type harness struct { prepMemo *memo.Memo testCat *testcat.Catalog optimizer xform.Optimizer + gf explain.PlanGistFactory } func newHarness(tb testing.TB, query benchQuery, schemas []string) *harness { @@ -943,12 +944,11 @@ func (h *harness) runSimple(tb testing.TB, query benchQuery, phase Phase) { tb.Fatalf("invalid phase %s for Simple", phase) } - var gf explain.PlanGistFactory - gf.Init(exec.StubFactory{}) + h.gf.Init(exec.StubFactory{}) root := execMemo.RootExpr() eb := execbuilder.New( context.Background(), - &gf, + &h.gf, &h.optimizer, execMemo, nil, /* catalog */ @@ -1001,12 +1001,11 @@ func (h *harness) runPrepared(tb testing.TB, phase Phase) { tb.Fatalf("invalid phase %s for Prepared", phase) } - var gf explain.PlanGistFactory - gf.Init(exec.StubFactory{}) + h.gf.Init(exec.StubFactory{}) root := execMemo.RootExpr() eb := execbuilder.New( context.Background(), - &gf, + &h.gf, &h.optimizer, execMemo, nil, /* catalog */ @@ -1779,9 +1778,9 @@ func BenchmarkExecBuild(b *testing.B) { execMemo := h.optimizer.Memo() root := execMemo.RootExpr() + var gf explain.PlanGistFactory b.Run(tc.query.name, func(b *testing.B) { for i := 0; i < b.N; i++ { - var gf explain.PlanGistFactory gf.Init(exec.StubFactory{}) eb := execbuilder.New( context.Background(), From a2f54657ad385e01cd1120eb77ae1b03698fcdbe Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 9 Jan 2025 15:08:04 +0100 Subject: [PATCH 014/126] cli: actually drain after decommission I do not know how this was ever supposed to work, but the old code can not have been intentional: it created a drain client but then did not consume from it. This had the effect of kicking off the drain, but ~immediately cancelling the context on the goroutine carrying it out on the decommissioning node. This PR actually waits for the drain to complete. There is a related issue here, though. You can't drain a node that isn't live, so attempting to decommission a node that's down will fail on the drain step. This is certainly true as of this PR, but should have been true before as well. Our docs[1] do not mention this rather large caveat at all, and it seems strange anyway; if the node is down why would you let the failing drain get in the way. Really the code ought to distinguish between the case of a live and dead node and react accordingly - this is not something this PR achieves. Fixes https://github.com/cockroachdb/cockroach/issues/138265. [1]: https://www.cockroachlabs.com/docs/v24.3/node-shutdown?filters=decommission#remove-nodes Epic: none Release note (ops change): the `node decommission` cli command now waits until the target node is drained before marking it as fully decommissioned. Previously, it would start drain but not wait, leaving the target node briefly in a state where it would be unable to communicate with the cluster but would still accept client requests (which would then hang or hit unexpected errors). --- pkg/cli/node.go | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/pkg/cli/node.go b/pkg/cli/node.go index 24da4a352bfa..fd32e7f74e17 100644 --- a/pkg/cli/node.go +++ b/pkg/cli/node.go @@ -8,6 +8,7 @@ package cli import ( "context" "fmt" + "io" "math" "os" "reflect" @@ -605,10 +606,25 @@ func runDecommissionNodeImpl( DoDrain: true, NodeId: targetNode.String(), } - if _, err = c.Drain(ctx, drainReq); err != nil { + stream, err := c.Drain(ctx, drainReq) + if err != nil { fmt.Fprintln(stderr) return errors.Wrapf(err, "while trying to drain n%d", targetNode) } + + // Consume responses until the stream ends (which signals drain + // completion). + for { + _, err := stream.Recv() + if err == io.EOF { + // Stream gracefully closed by other side. + break + } + if err != nil { + fmt.Fprintln(stderr) + return errors.Wrapf(err, "while trying to drain n%d", targetNode) + } + } } // Finally, mark the nodes as fully decommissioned. From 43b13bf4d34c8f2945d24de48bf84c628a47f4b7 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 9 Jan 2025 17:02:37 +0100 Subject: [PATCH 015/126] cli: improve logging --- pkg/cli/node.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pkg/cli/node.go b/pkg/cli/node.go index fd32e7f74e17..61e4c62e75ed 100644 --- a/pkg/cli/node.go +++ b/pkg/cli/node.go @@ -583,9 +583,11 @@ func runDecommissionNodeImpl( anyActive := false var replicaCount int64 + statusByNodeID := map[roachpb.NodeID]serverpb.DecommissionStatusResponse_Status{} for _, status := range resp.Status { anyActive = anyActive || status.Membership.Active() replicaCount += status.ReplicaCount + statusByNodeID[status.NodeID] = status } if !anyActive && replicaCount == 0 { @@ -601,6 +603,13 @@ func runDecommissionNodeImpl( ) continue } + if status, ok := statusByNodeID[targetNode]; !ok || !status.IsLive { + // Skip the draining step for the node serving the request, if it is a target node. + _, _ = fmt.Fprintf(stderr, + "skipping drain step for node n%d; it is not live\n", targetNode, + ) + continue + } drainReq := &serverpb.DrainRequest{ Shutdown: false, DoDrain: true, From fef82de9388939310ba825ca9731a0ef902eb393 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 9 Jan 2025 17:02:49 +0100 Subject: [PATCH 016/126] cli: improve logging --- pkg/cli/node.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/cli/node.go b/pkg/cli/node.go index 61e4c62e75ed..7fbe272e441d 100644 --- a/pkg/cli/node.go +++ b/pkg/cli/node.go @@ -597,8 +597,8 @@ func runDecommissionNodeImpl( for _, targetNode := range nodeIDs { if targetNode == localNodeID { // Skip the draining step for the node serving the request, if it is a target node. - log.Warningf(ctx, - "skipping drain step for node n%d; it is decommissioning and serving the request", + _, _ = fmt.Fprintf(stderr, + "skipping drain step for node n%d; it is decommissioning and serving the request\n", localNodeID, ) continue From fe56ffe13339201e50c150598006e7c9b10a5e13 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 9 Jan 2025 17:03:19 +0100 Subject: [PATCH 017/126] cli: fix typo --- pkg/cmd/roachtest/tests/decommission.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/decommission.go b/pkg/cmd/roachtest/tests/decommission.go index 3d2be685a071..7b6410e4f952 100644 --- a/pkg/cmd/roachtest/tests/decommission.go +++ b/pkg/cmd/roachtest/tests/decommission.go @@ -1056,7 +1056,7 @@ func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster) return err } - // Check to see if the node has been drained or decomissioned. + // Check to see if the node has been drained or decommissioned. // If not, queries should not fail. if err = run(decommNodeDB, `SHOW DATABASES`); err != nil { if strings.Contains(err.Error(), "not accepting clients") || // drained From fa40010608a403f1a0414ee4a808151cade88b45 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 9 Jan 2025 11:25:24 -0500 Subject: [PATCH 018/126] util/base64: use `bytes.Buffer` instead of `strings.Builder` in `Encoder` For our purposes in base64-encoding plan gists, using `bytes.Buffer` in `Encoder` causes fewer allocations, presumably because of a more aggressive growth algorithm. Release note: None --- pkg/util/base64/base64.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/util/base64/base64.go b/pkg/util/base64/base64.go index 507977f204d4..76d40299ab5e 100644 --- a/pkg/util/base64/base64.go +++ b/pkg/util/base64/base64.go @@ -6,8 +6,8 @@ package base64 import ( + "bytes" "encoding/base64" - "strings" ) // Encoder is a streaming encoder for base64 strings. It must be initialized @@ -21,7 +21,7 @@ import ( // handling has been removed for simplification. type Encoder struct { enc *base64.Encoding - sb strings.Builder + sb bytes.Buffer buf [3]byte // buffered data waiting to be encoded nbuf int8 // number of bytes in buf out [1024]byte // output buffer From e561bbf1dc3c44718c9438ee85f8509e23d00096 Mon Sep 17 00:00:00 2001 From: Ibrahim Kettaneh Date: Thu, 9 Jan 2025 15:12:58 -0500 Subject: [PATCH 019/126] kvserver: enable leader leases for StoreRangeMergeAbandonedFollowersAutomaticallyGarbageCollected This commit enables leader leases to run with TestStoreRangeMergeAbandonedFollowersAutomaticallyGarbageCollected. The test used to move the lease and stop Raft traffic without waiting for leadership to follow it. This could cause flakes with leader leases as if the replica doesn't know who the leader is, it might not be able to determine the lease status. References: #136806 Release Note: None --- pkg/kv/kvserver/client_merge_test.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index a4e59c1c52ea..6214b5de0949 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -2996,14 +2996,9 @@ func TestStoreRangeMergeAbandonedFollowersAutomaticallyGarbageCollected(t *testi defer log.Scope(t).Close(t) ctx := context.Background() - st := cluster.MakeTestingClusterSettings() - kvserver.OverrideLeaderLeaseMetamorphism(ctx, &st.SV) tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, - ServerArgs: base.TestServerArgs{ - Settings: st, - }, }) defer tc.Stopper().Stop(ctx) scratch := tc.ScratchRange(t) @@ -3027,6 +3022,13 @@ func TestStoreRangeMergeAbandonedFollowersAutomaticallyGarbageCollected(t *testi if !rhsRepl.OwnsValidLease(ctx, tc.Servers[2].Clock().NowAsClockTimestamp()) { return errors.New("store2 does not own valid lease for rhs range") } + + // This is important for leader leases to avoid a race between us stopping + // Raft traffic below, and Raft attempting to transfer the lease leadership + // to the leaseholder. + if rhsRepl.RaftStatus().ID != rhsRepl.RaftStatus().Lead { + return errors.New("store2 isn't the leader for rhs range") + } return nil }) From 26abaa1b91e68183b72db74bab647c9938f2a869 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Thu, 9 Jan 2025 17:05:08 -0500 Subject: [PATCH 020/126] roachtest: use thread safe rng in jobs/stress Fixes #138114 Release note: none --- pkg/cmd/roachtest/tests/jobs.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/jobs.go b/pkg/cmd/roachtest/tests/jobs.go index ba31d41ea8cc..126d22cf21b5 100644 --- a/pkg/cmd/roachtest/tests/jobs.go +++ b/pkg/cmd/roachtest/tests/jobs.go @@ -78,7 +78,7 @@ func runJobsStress(ctx context.Context, t test.Test, c cluster.Cluster) { // and adopts 10 jobs at a time. sqlDB.Exec(t, "SET CLUSTER SETTING jobs.registry.interval.adopt='5s'") - rng, seed := randutil.NewPseudoRand() + rng, seed := randutil.NewLockedPseudoRand() t.L().Printf("Rand seed: %d", seed) done := make(chan struct{}) From a9185e4f1bbd778f59353a2045efa9cf4bafaf5f Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Thu, 9 Jan 2025 18:29:14 -0500 Subject: [PATCH 021/126] server: add issue number for TODO Epic: none Release note: None --- pkg/server/span_stats_server.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/server/span_stats_server.go b/pkg/server/span_stats_server.go index 8b6b6d4cc6f1..4fc34385fab2 100644 --- a/pkg/server/span_stats_server.go +++ b/pkg/server/span_stats_server.go @@ -191,6 +191,7 @@ func collectSpanStatsResponses( // Logical values: take the values from the node that responded first. // TODO: This should really be read from the leaseholder. + // https://github.com/cockroachdb/cockroach/issues/138792 if _, ok := responses[spanStr]; !ok { res.SpanToStats[spanStr].TotalStats = spanStats.TotalStats res.SpanToStats[spanStr].RangeCount = spanStats.RangeCount From f42be073219dbf1d306c44e5711e3b4cf24f7363 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 10 Jan 2025 11:24:48 +0100 Subject: [PATCH 022/126] roachtest: fix network/authentication/nodes=4 For reasons, the test starts a three node cluster, then immediately stops it, then restarts n1 and {n2,n3} separately. It was previously restarting n1 first, followed by `{n2,n3}`. Due to recent change PR #138109, this no longer works since n1 doesn't have quorum and so won't signal SQL readiness. There's an ongoing discussion on whether this new behavior is desired, but either way, this PR changes the test to restart {n2,n3} first (which does have quorum, assuming we wait for full replication first, which we now do as well), followed by n1. Closes #138806. Epic: none Release note: None --- pkg/cmd/roachtest/tests/network.go | 33 ++++++++++++++++++------------ 1 file changed, 20 insertions(+), 13 deletions(-) diff --git a/pkg/cmd/roachtest/tests/network.go b/pkg/cmd/roachtest/tests/network.go index 1f36ccec6568..960ef573b80a 100644 --- a/pkg/cmd/roachtest/tests/network.go +++ b/pkg/cmd/roachtest/tests/network.go @@ -26,7 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/timeutil" - errors "github.com/cockroachdb/errors" + "github.com/cockroachdb/errors" _ "github.com/lib/pq" // register postgres driver "github.com/stretchr/testify/require" ) @@ -50,8 +50,13 @@ func runNetworkAuthentication(ctx context.Context, t test.Test, c cluster.Cluste // that they use coherent certs. settings := install.MakeClusterSettings() - // Don't create a backup schedule as this test shuts the cluster down immediately. - c.Start(ctx, t.L(), option.NewStartOpts(option.NoBackupSchedule), settings, serverNodes) + // Don't create a backup schedule in this test as the cluster won't be up + // long and we'll inject network issues. + // We wait for replication so that we can safely restart the cluster in two + // steps next. + c.Start( + ctx, t.L(), option.NewStartOpts(option.NoBackupSchedule, option.WaitForReplication()), settings, serverNodes, + ) require.NoError(t, c.StopE(ctx, t.L(), option.DefaultStopOpts(), serverNodes)) t.L().Printf("restarting nodes...") @@ -66,16 +71,18 @@ func runNetworkAuthentication(ctx context.Context, t test.Test, c cluster.Cluste // Currently, creating a scheduled backup at start fails, potentially due to // the induced network partition. Further investigation required to allow scheduled backups // to run on this test. - startOpts := option.NewStartOpts(option.NoBackupSchedule) - startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--locality=node=1", "--accept-sql-without-tls") - c.Start(ctx, t.L(), startOpts, settings, c.Node(1)) - - // See comment above about env vars. - // "--env=COCKROACH_SCAN_INTERVAL=200ms", - // "--env=COCKROACH_SCAN_MAX_IDLE_TIME=20ms", - startOpts = option.NewStartOpts(option.NoBackupSchedule) - startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--locality=node=other", "--accept-sql-without-tls") - c.Start(ctx, t.L(), startOpts, settings, c.Range(2, n-1)) + { + // We start n2+ first so that there's quorum. + startOpts := option.NewStartOpts(option.NoBackupSchedule) + startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--locality=node=other", "--accept-sql-without-tls") + c.Start(ctx, t.L(), startOpts, settings, c.Range(2, n-1)) + } + { + // Now start n1. + startOpts := option.NewStartOpts(option.NoBackupSchedule) + startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--locality=node=1", "--accept-sql-without-tls") + c.Start(ctx, t.L(), startOpts, settings, c.Node(1)) + } t.L().Printf("retrieving server addresses...") serverUrls, err := c.InternalPGUrl(ctx, t.L(), serverNodes, roachprod.PGURLOptions{Auth: install.AuthUserPassword}) From c326029525d4bf90b5f6bc453e2b566e6f6da1a0 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 10 Jan 2025 11:47:56 +0100 Subject: [PATCH 023/126] roachtest: small refactor --- pkg/cmd/roachtest/tests/decommission.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/pkg/cmd/roachtest/tests/decommission.go b/pkg/cmd/roachtest/tests/decommission.go index 7b6410e4f952..04d79c5d875d 100644 --- a/pkg/cmd/roachtest/tests/decommission.go +++ b/pkg/cmd/roachtest/tests/decommission.go @@ -1021,10 +1021,6 @@ func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster) require.NoError(t, err) } - // Connect to node 4 (the target node of the decommission). - decommNodeDB := c.Conn(ctx, t.L(), decommNodeID) - defer decommNodeDB.Close() - // Decommission node 4 and poll its status during the decommission. var ( maxAttempts = 50 @@ -1058,6 +1054,9 @@ func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster) // Check to see if the node has been drained or decommissioned. // If not, queries should not fail. + // Connect to node 4 (the target node of the decommission). + decommNodeDB := c.Conn(ctx, t.L(), decommNodeID) + defer decommNodeDB.Close() if err = run(decommNodeDB, `SHOW DATABASES`); err != nil { if strings.Contains(err.Error(), "not accepting clients") || // drained strings.Contains(err.Error(), "node is decommissioned") { // decommissioned From a72cc4f5842c04873a5940acb80eec71326ccf66 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 10 Jan 2025 12:02:44 +0100 Subject: [PATCH 024/126] roachtest: decommission/drains -> decommission/drains/{live,dead} - rename `decommission/drains` to `decommission/drains/alive` - add `decommission/drains/dead` flavor - run these weekly instead of daily (we have enough other decom tests and since I'm adding one, we can also clamp down a bit) - remove an old workaround that also accepted errors that should be impossible now that we properly wait for drain. Epic: none --- pkg/cmd/roachtest/tests/decommission.go | 58 +++++++++++++++++-------- 1 file changed, 41 insertions(+), 17 deletions(-) diff --git a/pkg/cmd/roachtest/tests/decommission.go b/pkg/cmd/roachtest/tests/decommission.go index 04d79c5d875d..cad4938104bd 100644 --- a/pkg/cmd/roachtest/tests/decommission.go +++ b/pkg/cmd/roachtest/tests/decommission.go @@ -75,17 +75,25 @@ func registerDecommission(r registry.Registry) { } { numNodes := 4 - r.Add(registry.TestSpec{ - Name: "decommission/drains", - Owner: registry.OwnerKV, - Cluster: r.MakeClusterSpec(numNodes), - CompatibleClouds: registry.AllExceptAWS, - Suites: registry.Suites(registry.Nightly), - Leases: registry.MetamorphicLeases, - Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { - runDecommissionDrains(ctx, t, c) - }, - }) + for _, dead := range []bool{false, true} { + name := "decommission/drains" + if dead { + name += "/dead" + } else { + name += "/alive" + } + r.Add(registry.TestSpec{ + Name: name, + Owner: registry.OwnerKV, + Cluster: r.MakeClusterSpec(numNodes), + CompatibleClouds: registry.AllExceptAWS, + Suites: registry.Suites(registry.Weekly), + Leases: registry.MetamorphicLeases, + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + runDecommissionDrains(ctx, t, c, dead) + }, + }) + } } { numNodes := 6 @@ -991,7 +999,7 @@ func runDecommissionRandomized(ctx context.Context, t test.Test, c cluster.Clust // the end of decommissioning. The test cluster contains 4 nodes and the fourth // node is decommissioned. While the decommissioning node has open SQL // connections, queries should never fail. -func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster) { +func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster, dead bool) { var ( numNodes = 4 pinnedNodeID = 1 @@ -1032,16 +1040,25 @@ func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster) // The expected output of decommission while the node is about to be drained/is draining. expReplicasTransferred = [][]string{ decommissionHeader, - {strconv.Itoa(decommNodeID), "true|false", "0", "true", "decommissioning", "false", "ready", "0"}, + {strconv.Itoa(decommNodeID), "true|false", "0", "true", "decommissioning", "false", ".*", "0"}, decommissionFooter, } // The expected output of decommission once the node is finally marked as "decommissioned." expDecommissioned = [][]string{ decommissionHeader, - {strconv.Itoa(decommNodeID), "true|false", "0", "true", "decommissioned", "false", "ready", "0"}, + {strconv.Itoa(decommNodeID), "true|false", "0", "true", "decommissioned", "false", ".*", "0"}, decommissionFooter, } ) + if dead { + t.Status(fmt.Sprintf("stopping node %d and waiting for it to be recognized as dead", decommNodeID)) + c.Stop(ctx, t.L(), option.DefaultStopOpts(), decommNode) + // This should reliably result in the node being perceived as non-live from + // this point on. If the node were still "down but live" when decommission + // finishes, we'd try to drain a live node and get an error (since it can't + // be reached any more). + time.Sleep(15 * time.Second) + } t.Status(fmt.Sprintf("decommissioning node %d", decommNodeID)) e := retry.WithMaxAttempts(ctx, retryOpts, maxAttempts, func() error { o, err := h.decommission(ctx, decommNode, pinnedNodeID, "--wait=none", "--format=csv") @@ -1052,15 +1069,22 @@ func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster) return err } + // When the target node is dead in this test configuration, the draining + // step is moot. If the target node is alive, the last decommission + // invocation should have drained it, which we verify below. + + if dead { + return nil + } + // Check to see if the node has been drained or decommissioned. // If not, queries should not fail. // Connect to node 4 (the target node of the decommission). decommNodeDB := c.Conn(ctx, t.L(), decommNodeID) defer decommNodeDB.Close() if err = run(decommNodeDB, `SHOW DATABASES`); err != nil { - if strings.Contains(err.Error(), "not accepting clients") || // drained - strings.Contains(err.Error(), "node is decommissioned") { // decommissioned - return nil + if strings.Contains(err.Error(), "not accepting clients") { + return nil // success (drained) } t.Fatal(err) } From ba01cb2ac913eac2a67ba17303fd04d33b300842 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Thu, 9 Jan 2025 12:21:45 +0000 Subject: [PATCH 025/126] roachprod: configure chrony on Azure Both GCE & AWS sets up chrony for clock synchronisation. The Azure Ubuntu VM image has chrony set up by default, but its configuration is slightly different from the config we use in GCS & AWS. The Azure config also opts to use a PTP Clock Source that it configures by setting the `refclock` in the crony config (See: https://learn.microsoft.com/en-us/azure/virtual-machines/linux/time-sync). This source seems to be unreliable, considering we have had a few clock sync issues that prompted this change. Unlike GCS and AWS, Azure does not provide a specific NTP server. Hence, we opt to use Google's time-servers. And as with the other startup scripts we also do a crony waitsync to ensure the clock is in sync before proceeding. Our docs also suggests disabling the Hyper-V Time Synchronization device (See: https://www.cockroachlabs.com/docs/stable/deploy-cockroachdb-on-microsoft-azure#step-3-synchronize-clocks). This has been added as part of the start-up script. Informs: #138726 Release note: None Epic: None --- pkg/roachprod/vm/azure/utils.go | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/pkg/roachprod/vm/azure/utils.go b/pkg/roachprod/vm/azure/utils.go index dd6227fae937..04cf8c24aec9 100644 --- a/pkg/roachprod/vm/azure/utils.go +++ b/pkg/roachprod/vm/azure/utils.go @@ -103,6 +103,37 @@ systemctl stop unattended-upgrades sudo rm -rf /var/log/unattended-upgrades apt-get purge -y unattended-upgrades +# Disable Hyper-V Time Synchronization device +# See https://www.cockroachlabs.com/docs/stable/deploy-cockroachdb-on-microsoft-azure#step-3-synchronize-clocks +curl -O https://raw.githubusercontent.com/torvalds/linux/master/tools/hv/lsvmbus +ts_dev_id=$(python3 lsvmbus -vv | grep -w "Time Synchronization" -A 3 | grep "Device_ID" | awk -F '[{}]' '{print $2}') +echo $ts_dev_id | sudo tee /sys/bus/vmbus/drivers/hv_utils/unbind + +# Override the chrony config. In particular, +# log aggressively when clock is adjusted (0.01s) +# and exclusively use a single time server. +sudo cat < /etc/chrony/chrony.conf +keyfile /etc/chrony/chrony.keys +commandkey 1 +driftfile /var/lib/chrony/chrony.drift +log tracking measurements statistics +logdir /var/log/chrony +maxupdateskew 100.0 +dumponexit +dumpdir /var/lib/chrony +logchange 0.01 +hwclockfile /etc/adjtime +rtcsync +server time1.google.com iburst +server time2.google.com iburst +server time3.google.com iburst +server time4.google.com iburst +makestep 0.1 3 +EOF + +sudo /etc/init.d/chrony restart +sudo chronyc -a waitsync 30 0.01 | sudo tee -a /root/chrony.log + # Enable core dumps cat < /etc/security/limits.d/core_unlimited.conf * soft core unlimited From 87d99d96b78fdf4fdf99a3a7db8be5a151981238 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 6 Jan 2025 13:07:57 +0100 Subject: [PATCH 026/126] go get google.golang.org/grpc@v1.57.2 --- go.mod | 5 +++-- go.sum | 6 ++++++ 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 6e8d3f08cfc8..7c77cb814199 100644 --- a/go.mod +++ b/go.mod @@ -38,8 +38,8 @@ require ( github.com/google/pprof v0.0.0-20240227163752-401108e1b7e7 github.com/google/uuid v1.6.0 // indirect google.golang.org/api v0.114.0 - google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 - google.golang.org/grpc v1.56.3 + google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54 + google.golang.org/grpc v1.57.2 google.golang.org/protobuf v1.35.1 ) @@ -448,6 +448,7 @@ require ( golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect gonum.org/v1/plot v0.14.0 // indirect google.golang.org/appengine v1.6.7 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19 // indirect gopkg.in/square/go-jose.v2 v2.5.1 // indirect ) diff --git a/go.sum b/go.sum index db552ac6f7a9..40bdfb3235ce 100644 --- a/go.sum +++ b/go.sum @@ -3145,6 +3145,10 @@ google.golang.org/genproto v0.0.0-20210828152312-66f60bf46e71/go.mod h1:eFjDcFEc google.golang.org/genproto v0.0.0-20210903162649-d08c68adba83/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 h1:KpwkzHKEF7B9Zxg18WzOa7djJ+Ha5DzthMyZYQfEn2A= google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= +google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54 h1:9NWlQfY2ePejTmfwUH1OWwmznFa+0kKcHGPDvcPza9M= +google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54/go.mod h1:zqTuNwFlFRsw5zIts5VnzLQxSRqh+CGOTVMlYbY0Eyk= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19 h1:0nDDozoAU19Qb2HwhXadU8OcsiO/09cnTqhUtq2MEOM= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= google.golang.org/grpc v0.0.0-20160317175043-d3ddb4469d5a/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v0.0.0-20170208002647-2a6bf6142e96/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.8.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= @@ -3185,6 +3189,8 @@ google.golang.org/grpc v1.41.0-dev.0.20210907181116-2f3355d2244e/go.mod h1:U3l9u google.golang.org/grpc v1.42.0/go.mod h1:k+4IHHFw41K8+bbowsex27ge2rCb65oeWqe4jJ590SU= google.golang.org/grpc v1.56.3 h1:8I4C0Yq1EjstUzUJzpcRVbuYA2mODtEmpWiQoN/b2nc= google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= +google.golang.org/grpc v1.57.2 h1:uw37EN34aMFFXB2QPW7Tq6tdTbind1GpRxw5aOX3a5k= +google.golang.org/grpc v1.57.2/go.mod h1:Sd+9RMTACXwmub0zcNY2c4arhtrbBYD1AUHI/dt16Mo= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/grpc/examples v0.0.0-20210324172016-702608ffae4d h1:CJP00gEaaYNJuaTXEg456rWNC1oUOfiAiUjuwyAhEmM= google.golang.org/grpc/examples v0.0.0-20210324172016-702608ffae4d/go.mod h1:Ly7ZA/ARzg8fnPU9TyZIxoz33sEUuWX7txiqs8lPTgE= From 59e987a9f0488e45826ecf1c802510b452145667 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 10 Jan 2025 15:59:00 +0100 Subject: [PATCH 027/126] DEPS: go get google.golang.org/genproto/googleapis/bytestream@2805bf891e895db98bfa8bf3202ceb2862953e06 --- go.mod | 2 ++ go.sum | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/go.mod b/go.mod index 7c77cb814199..ac19f06f4298 100644 --- a/go.mod +++ b/go.mod @@ -448,6 +448,8 @@ require ( golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect gonum.org/v1/plot v0.14.0 // indirect google.golang.org/appengine v1.6.7 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9 // indirect + google.golang.org/genproto/googleapis/bytestream v0.0.0-20230525234009-2805bf891e89 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19 // indirect gopkg.in/square/go-jose.v2 v2.5.1 // indirect ) diff --git a/go.sum b/go.sum index 40bdfb3235ce..5fa134554f65 100644 --- a/go.sum +++ b/go.sum @@ -3147,6 +3147,10 @@ google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 h1:KpwkzHKEF7B9Zxg google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54 h1:9NWlQfY2ePejTmfwUH1OWwmznFa+0kKcHGPDvcPza9M= google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54/go.mod h1:zqTuNwFlFRsw5zIts5VnzLQxSRqh+CGOTVMlYbY0Eyk= +google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9 h1:m8v1xLLLzMe1m5P+gCTF8nJB9epwZQUBERm20Oy1poQ= +google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= +google.golang.org/genproto/googleapis/bytestream v0.0.0-20230525234009-2805bf891e89 h1:VFsw3MyBfHHuzZ4EAfzcfpjm3LKUKegxBnLYdK4GISo= +google.golang.org/genproto/googleapis/bytestream v0.0.0-20230525234009-2805bf891e89/go.mod h1:iNmxyobPsX4ATDNyZE1+2xFUvAs/81HLbg8fL12YoGs= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19 h1:0nDDozoAU19Qb2HwhXadU8OcsiO/09cnTqhUtq2MEOM= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= google.golang.org/grpc v0.0.0-20160317175043-d3ddb4469d5a/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= From 00f75802ab71e1a3a6bbd384ffd4f1361c7da386 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 10 Jan 2025 15:59:22 +0100 Subject: [PATCH 028/126] DEPS: go mod tidy -v --- go.mod | 2 +- go.sum | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/go.mod b/go.mod index ac19f06f4298..a63175a08704 100644 --- a/go.mod +++ b/go.mod @@ -253,6 +253,7 @@ require ( golang.org/x/term v0.27.0 golang.org/x/tools/go/vcs v0.1.0-deprecated gonum.org/v1/gonum v0.15.1 + google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 honnef.co/go/tools v0.4.5 @@ -448,7 +449,6 @@ require ( golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect gonum.org/v1/plot v0.14.0 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9 // indirect google.golang.org/genproto/googleapis/bytestream v0.0.0-20230525234009-2805bf891e89 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19 // indirect gopkg.in/square/go-jose.v2 v2.5.1 // indirect diff --git a/go.sum b/go.sum index 5fa134554f65..25013d2226cd 100644 --- a/go.sum +++ b/go.sum @@ -3143,8 +3143,6 @@ google.golang.org/genproto v0.0.0-20210813162853-db860fec028c/go.mod h1:cFeNkxwy google.golang.org/genproto v0.0.0-20210821163610-241b8fcbd6c8/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= google.golang.org/genproto v0.0.0-20210828152312-66f60bf46e71/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= google.golang.org/genproto v0.0.0-20210903162649-d08c68adba83/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= -google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 h1:KpwkzHKEF7B9Zxg18WzOa7djJ+Ha5DzthMyZYQfEn2A= -google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54 h1:9NWlQfY2ePejTmfwUH1OWwmznFa+0kKcHGPDvcPza9M= google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54/go.mod h1:zqTuNwFlFRsw5zIts5VnzLQxSRqh+CGOTVMlYbY0Eyk= google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9 h1:m8v1xLLLzMe1m5P+gCTF8nJB9epwZQUBERm20Oy1poQ= @@ -3191,8 +3189,6 @@ google.golang.org/grpc v1.39.1/go.mod h1:PImNr+rS9TWYb2O4/emRugxiyHZ5JyHW5F+RPnD google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= google.golang.org/grpc v1.41.0-dev.0.20210907181116-2f3355d2244e/go.mod h1:U3l9uK9J0sini8mHphKoXyaqDA/8VyGnDee1zzIUK6k= google.golang.org/grpc v1.42.0/go.mod h1:k+4IHHFw41K8+bbowsex27ge2rCb65oeWqe4jJ590SU= -google.golang.org/grpc v1.56.3 h1:8I4C0Yq1EjstUzUJzpcRVbuYA2mODtEmpWiQoN/b2nc= -google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= google.golang.org/grpc v1.57.2 h1:uw37EN34aMFFXB2QPW7Tq6tdTbind1GpRxw5aOX3a5k= google.golang.org/grpc v1.57.2/go.mod h1:Sd+9RMTACXwmub0zcNY2c4arhtrbBYD1AUHI/dt16Mo= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= From 4e3b377d831c8dede8f6adbd64bd214df9bd87c4 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 10 Jan 2025 16:01:00 +0100 Subject: [PATCH 029/126] DEPS: ./dev gen --mirror --- DEPS.bzl | 42 ++++++++++++++++++++++++++----- build/bazelutil/distdir_files.bzl | 7 ++++-- 2 files changed, 41 insertions(+), 8 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 5ed48b17c1e7..41486147ec4b 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -11539,20 +11539,50 @@ def go_deps(): name = "org_golang_google_genproto", build_file_proto_mode = "disable_global", importpath = "google.golang.org/genproto", - sha256 = "28f0317e6948788a33c07698109005675062f0203ed06bc866350a575bc974bf", - strip_prefix = "google.golang.org/genproto@v0.0.0-20230410155749-daa745c078e1", + sha256 = "dd3a50a0b27c99b52c9bbcf1ed54529b8d95f97c11d7f50d734a2592f2caa766", + strip_prefix = "google.golang.org/genproto@v0.0.0-20230526161137-0005af68ea54", urls = [ - "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20230410155749-daa745c078e1.zip", + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20230526161137-0005af68ea54.zip", + ], + ) + go_repository( + name = "org_golang_google_genproto_googleapis_api", + build_file_proto_mode = "disable_global", + importpath = "google.golang.org/genproto/googleapis/api", + sha256 = "c0a71d9865c3e65e58006030a7f54a426c485fc28a3cc14485360ac210fbf922", + strip_prefix = "google.golang.org/genproto/googleapis/api@v0.0.0-20230525234035-dd9d682886f9", + urls = [ + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20230525234035-dd9d682886f9.zip", + ], + ) + go_repository( + name = "org_golang_google_genproto_googleapis_bytestream", + build_file_proto_mode = "disable_global", + importpath = "google.golang.org/genproto/googleapis/bytestream", + sha256 = "92c879d91de794eda21e38ef0e97b89c1015582d25e665d18cf0acc7d1c7ba84", + strip_prefix = "google.golang.org/genproto/googleapis/bytestream@v0.0.0-20230525234009-2805bf891e89", + urls = [ + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20230525234009-2805bf891e89.zip", + ], + ) + go_repository( + name = "org_golang_google_genproto_googleapis_rpc", + build_file_proto_mode = "disable_global", + importpath = "google.golang.org/genproto/googleapis/rpc", + sha256 = "01b51779ca4ac5ff536b13b9bee9ef4df36b14663d66e27fa7abe0075a0d8e80", + strip_prefix = "google.golang.org/genproto/googleapis/rpc@v0.0.0-20230525234030-28d5490b6b19", + urls = [ + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20230525234030-28d5490b6b19.zip", ], ) go_repository( name = "org_golang_google_grpc", build_file_proto_mode = "disable_global", importpath = "google.golang.org/grpc", - sha256 = "7f9b83cd9dbe99783061160583dae09dd8f64187d890c51be52cc8e3bd0dcb56", - strip_prefix = "google.golang.org/grpc@v1.56.3", + sha256 = "4cabfaa890b25757939e2221b93fe22b3097a0b83ffb331e922143a68f3da252", + strip_prefix = "google.golang.org/grpc@v1.57.2", urls = [ - "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/grpc/org_golang_google_grpc-v1.56.3.zip", + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/grpc/org_golang_google_grpc-v1.57.2.zip", ], ) go_repository( diff --git a/build/bazelutil/distdir_files.bzl b/build/bazelutil/distdir_files.bzl index d5596ae18a41..3a17419d7205 100644 --- a/build/bazelutil/distdir_files.bzl +++ b/build/bazelutil/distdir_files.bzl @@ -1120,10 +1120,13 @@ DISTDIR_FILES = { "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/api/org_golang_google_api-v0.114.0.zip": "42c62aaba1d76efede08c70d8aef7889c5c8ee9c9c4f1e7c455b07838cabb785", "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/appengine/org_golang_google_appengine-v1.6.7.zip": "79f80dfac18681788f1414e21a4a7734eff4cdf992070be9163103eb8d9f92cd", "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/cloud/org_golang_google_cloud-v0.0.0-20151119220103-975617b05ea8.zip": "b1d5595a11b88273665d35d4316edbd4545731c979d046c82844fafef2039c2a", - "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20230410155749-daa745c078e1.zip": "28f0317e6948788a33c07698109005675062f0203ed06bc866350a575bc974bf", + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20230525234035-dd9d682886f9.zip": "c0a71d9865c3e65e58006030a7f54a426c485fc28a3cc14485360ac210fbf922", + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20230525234009-2805bf891e89.zip": "92c879d91de794eda21e38ef0e97b89c1015582d25e665d18cf0acc7d1c7ba84", + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20230525234030-28d5490b6b19.zip": "01b51779ca4ac5ff536b13b9bee9ef4df36b14663d66e27fa7abe0075a0d8e80", + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20230526161137-0005af68ea54.zip": "dd3a50a0b27c99b52c9bbcf1ed54529b8d95f97c11d7f50d734a2592f2caa766", "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/grpc/cmd/protoc-gen-go-grpc/org_golang_google_grpc_cmd_protoc_gen_go_grpc-v1.1.0.zip": "13877d86cbfa30bde4d62fef2bc58dd56377dcb502c16cf78197f6934193009a", "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/grpc/examples/org_golang_google_grpc_examples-v0.0.0-20210324172016-702608ffae4d.zip": "f5cad7b05a93557c91864a02890a35c6bc5c394897222978cff2b880a78f7a11", - "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/grpc/org_golang_google_grpc-v1.56.3.zip": "7f9b83cd9dbe99783061160583dae09dd8f64187d890c51be52cc8e3bd0dcb56", + "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/grpc/org_golang_google_grpc-v1.57.2.zip": "4cabfaa890b25757939e2221b93fe22b3097a0b83ffb331e922143a68f3da252", "https://storage.googleapis.com/cockroach-godeps/gomod/google.golang.org/protobuf/org_golang_google_protobuf-v1.35.1.zip": "5a27ed9bbe348c7435d91f699af976d0f7dc40c324542e4f41076a425d9e793e", "https://storage.googleapis.com/cockroach-godeps/gomod/gopkg.in/DataDog/dd-trace-go.v1/in_gopkg_datadog_dd_trace_go_v1-v1.17.0.zip": "2ebcc818df0b2d560a61037da4492ae7effbaed67de94339a1d3a72728d2cb09", "https://storage.googleapis.com/cockroach-godeps/gomod/gopkg.in/airbrake/gobrake.v2/in_gopkg_airbrake_gobrake_v2-v2.0.9.zip": "2db903664908e5a9afafefba94821b9579bbf271e2929c1f0b7b1fdd23f7bbcf", From 3708ee527528cc523971eaff29a9915fffc8d36b Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 10 Jan 2025 16:43:32 +0100 Subject: [PATCH 030/126] DEPS: add resolve hints and update packages A few former subpackages are now proper modules, which requires some gazelle/bzl wrangling. This caused a few spurious diffs where unrelated lines got reordered. I didn't do this manually. --- BUILD.bazel | 4 +++- build/patches/com_github_buchgr_bazel_remote.patch | 8 ++++---- .../patches/com_github_grpc_ecosystem_grpc_gateway.patch | 4 ++-- build/patches/go_googleapis.patch | 2 +- pkg/server/serverpb/BUILD.bazel | 4 ++-- pkg/ts/tspb/BUILD.bazel | 6 +++--- 6 files changed, 15 insertions(+), 13 deletions(-) diff --git a/BUILD.bazel b/BUILD.bazel index 92c7601bea93..fdccedd45eaf 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -45,7 +45,7 @@ exports_files([ # gazelle:resolve proto proto gogoproto/gogo.proto @com_github_gogo_protobuf//gogoproto:gogo_proto # gazelle:resolve proto go gogoproto/gogo.proto @com_github_gogo_protobuf//gogoproto # gazelle:resolve proto proto google/api/annotations.proto @go_googleapis//google/api:annotations_proto -# gazelle:resolve proto go google/api/annotations.proto @org_golang_google_genproto//googleapis/api/annotations:go_default_library +# gazelle:resolve proto go google/api/annotations.proto @org_golang_google_genproto_googleapis_api//annotations:go_default_library # gazelle:resolve proto io/prometheus/client/metrics.proto @com_github_prometheus_client_model//io/prometheus/client:io_prometheus_client_proto # gazelle:resolve proto go io/prometheus/client/metrics.proto @com_github_prometheus_client_model//go # gazelle:resolve go github.com/prometheus/client_model/go @com_github_prometheus_client_model//go @@ -57,6 +57,8 @@ exports_files([ # gazelle:resolve go google.golang.org/genproto/googleapis/pubsub/v1 @org_golang_google_genproto//googleapis/pubsub/v1:pubsub # gazelle:resolve go google.golang.org/genproto/googleapis/cloud/kms/v1 @org_golang_google_genproto//googleapis/cloud/kms/v1:kms # gazelle:resolve go google.golang.org/genproto/googleapis/devtools/build/v1 @org_golang_google_genproto//googleapis/devtools/build/v1:build +# gazelle:resolve go google.golang.org/genproto/googleapis/api/httpbody @org_golang_google_genproto_googleapis_api//httpbody:go_default_library +# gazelle:resolve go google.golang.org/genproto/googleapis/api/annotations @org_golang_google_genproto_googleapis_api//annotations:go_default_library # See pkg/kv/kvpb/gen/BUILD.bazel for more details. # diff --git a/build/patches/com_github_buchgr_bazel_remote.patch b/build/patches/com_github_buchgr_bazel_remote.patch index f83782dc9e1a..5bf4f52a070a 100644 --- a/build/patches/com_github_buchgr_bazel_remote.patch +++ b/build/patches/com_github_buchgr_bazel_remote.patch @@ -113,8 +113,8 @@ diff -urN a/genproto/build/bazel/remote/asset/v1/BUILD.bazel b/genproto/build/ba - "@go_googleapis//google/rpc:status_go_proto", - "@io_bazel_rules_go//proto/wkt:duration_go_proto", - "@io_bazel_rules_go//proto/wkt:timestamp_go_proto", -+ "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", -+ "@org_golang_google_genproto//googleapis/rpc/status:go_default_library", ++ "@org_golang_google_genproto_googleapis_api//annotations:go_default_library", ++ "@org_golang_google_genproto_googleapis_rpc//status:go_default_library", + "@com_github_golang_protobuf//ptypes/duration:go_default_library", + "@com_github_golang_protobuf//ptypes/timestamp:go_default_library", "@org_golang_google_grpc//:go_default_library", @@ -133,9 +133,9 @@ diff -urN a/genproto/build/bazel/remote/execution/v2/BUILD.bazel b/genproto/buil - "@io_bazel_rules_go//proto/wkt:duration_go_proto", - "@io_bazel_rules_go//proto/wkt:timestamp_go_proto", - "@io_bazel_rules_go//proto/wkt:wrappers_go_proto", -+ "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", ++ "@org_golang_google_genproto_googleapis_api//annotations:go_default_library", + "@org_golang_google_genproto//googleapis/longrunning:go_default_library", -+ "@org_golang_google_genproto//googleapis/rpc/status:go_default_library", ++ "@org_golang_google_genproto_googleapis_rpc//status:go_default_library", + "@com_github_golang_protobuf//ptypes/duration:go_default_library", + "@com_github_golang_protobuf//ptypes/timestamp:go_default_library", + "@com_github_golang_protobuf//ptypes/wrappers:go_default_library", diff --git a/build/patches/com_github_grpc_ecosystem_grpc_gateway.patch b/build/patches/com_github_grpc_ecosystem_grpc_gateway.patch index a2256cfcb0df..ba38eda61967 100644 --- a/build/patches/com_github_grpc_ecosystem_grpc_gateway.patch +++ b/build/patches/com_github_grpc_ecosystem_grpc_gateway.patch @@ -40,7 +40,7 @@ index 02aa7d2a8..fd922a156 100644 + "@com_github_golang_protobuf//protoc-gen-go/descriptor:go_default_library", "@io_bazel_rules_go//proto/wkt:compiler_plugin_go_proto", - "@io_bazel_rules_go//proto/wkt:descriptor_go_proto", -+ "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", ++ "@org_golang_google_genproto_googleapis_api//annotations:go_default_library", ], ) @@ -65,7 +65,7 @@ index c4d18f624..41d5319fb 100644 + "@com_github_golang_protobuf//ptypes/duration:go_default_library", + "@com_github_golang_protobuf//ptypes/timestamp:go_default_library", + "@com_github_golang_protobuf//ptypes/wrappers:go_default_library", -+ "@org_golang_google_genproto//googleapis/api/httpbody:go_default_library", ++ "@org_golang_google_genproto_googleapis_api//httpbody:go_default_library", + "@org_golang_google_genproto//protobuf/field_mask:go_default_library", "@org_golang_google_grpc//codes:go_default_library", "@org_golang_google_grpc//grpclog:go_default_library", diff --git a/build/patches/go_googleapis.patch b/build/patches/go_googleapis.patch index 6ebc6706fd9a..a9181d312e78 100644 --- a/build/patches/go_googleapis.patch +++ b/build/patches/go_googleapis.patch @@ -10,7 +10,7 @@ diff -urN a/google/cloud/kms/v1/BUILD.bazel b/google/cloud/kms/v1/BUILD.bazel + "@com_github_golang_protobuf//ptypes/duration:go_default_library", + "@com_github_golang_protobuf//ptypes/timestamp:go_default_library", + "@com_github_golang_protobuf//ptypes/wrappers:go_default_library", -+ "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", ++ "@org_golang_google_genproto_googleapis_api//annotations:go_default_library", + "@org_golang_google_genproto//protobuf/field_mask:go_default_library", ], ) diff --git a/pkg/server/serverpb/BUILD.bazel b/pkg/server/serverpb/BUILD.bazel index d08d869dc9b8..51c1d3821b85 100644 --- a/pkg/server/serverpb/BUILD.bazel +++ b/pkg/server/serverpb/BUILD.bazel @@ -1,8 +1,8 @@ # gazelle:go_grpc_compilers //pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_grpc_compiler, @com_github_grpc_ecosystem_grpc_gateway//protoc-gen-grpc-gateway:go_gen_grpc_gateway -load("@rules_proto//proto:defs.bzl", "proto_library") load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +load("@rules_proto//proto:defs.bzl", "proto_library") proto_library( name = "serverpb_proto", @@ -72,7 +72,6 @@ go_proto_library( # NB: The grpc-gateway compiler injects a dependency on the descriptor # package that Gazelle isn't prepared to deal with. "@com_github_golang_protobuf//descriptor:go_default_library_gen", # keep - "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", "//pkg/build", "//pkg/clusterversion", "//pkg/config/zonepb", @@ -95,6 +94,7 @@ go_proto_library( "//pkg/util/log/logpb", "//pkg/util/metric", "//pkg/util/tracing/tracingpb", + "@org_golang_google_genproto_googleapis_api//annotations:go_default_library", ], ) diff --git a/pkg/ts/tspb/BUILD.bazel b/pkg/ts/tspb/BUILD.bazel index b349a741042c..9963241b93d0 100644 --- a/pkg/ts/tspb/BUILD.bazel +++ b/pkg/ts/tspb/BUILD.bazel @@ -1,8 +1,8 @@ # gazelle:go_grpc_compilers //pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_grpc_compiler, @com_github_grpc_ecosystem_grpc_gateway//protoc-gen-grpc-gateway:go_gen_grpc_gateway -load("@rules_proto//proto:defs.bzl", "proto_library") -load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +load("@rules_proto//proto:defs.bzl", "proto_library") go_library( name = "tspb", @@ -38,6 +38,6 @@ go_proto_library( "//pkg/roachpb", "@com_github_gogo_protobuf//gogoproto", "@com_github_golang_protobuf//descriptor:go_default_library_gen", # keep - "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", + "@org_golang_google_genproto_googleapis_api//annotations:go_default_library", ], ) From f42bceafbff22dd4079aa7f942c241edb3c40580 Mon Sep 17 00:00:00 2001 From: Matt Spilchen Date: Fri, 10 Jan 2025 15:29:11 -0400 Subject: [PATCH 031/126] sql/schemachanger: Implement CREATE/DROP POLICY in the DSC Prior to this commit, attempts to create or drop a policy succeeded but did not write any metadata to the table descriptor. This commit introduces the first step in persisting policy metadata by storing the policy name. Future PRs will handle storing additional policy details. The policy name is now stored in a new field within the table descriptor, and each policy is assigned a unique ID specific to the table. This commit also adds elements to the DSC to support adding and removing policy information in the descriptor. Support in the legacy schema changer has been intentionally left out. Epic: CRDB-11724 Informs #136696 Release note: none --- .../backup_base_generated_test.go | 56 ++++ pkg/cli/testdata/doctor/test_recreate_zipdir | 2 +- .../testdata/doctor/test_recreate_zipdir-json | 12 +- pkg/sql/catalog/bootstrap/testdata/testdata | 252 +++++++++--------- pkg/sql/catalog/descpb/structured.go | 3 + pkg/sql/catalog/descpb/structured.proto | 22 +- pkg/sql/catalog/descriptor.go | 5 + pkg/sql/catalog/table_elements.go | 13 + pkg/sql/catalog/tabledesc/table.go | 6 + pkg/sql/catalog/tabledesc/validate.go | 36 +++ pkg/sql/catalog/tabledesc/validate_test.go | 65 +++++ .../testdata/logic_test/row_level_security | 103 +++++++ .../generated_test.go | 7 - .../tests/local-mixed-24.3/generated_test.go | 7 - .../opt/exec/execbuilder/testdata/show_trace | 12 +- pkg/sql/parser/sql.y | 6 +- .../schemachanger/scbuild/builder_state.go | 53 ++++ pkg/sql/schemachanger/scbuild/builder_test.go | 1 + .../scbuild/internal/scbuildstmt/BUILD.bazel | 1 + .../internal/scbuildstmt/create_policy.go | 37 ++- .../internal/scbuildstmt/dependencies.go | 7 + .../internal/scbuildstmt/drop_policy.go | 35 ++- .../scbuild/internal/scbuildstmt/helpers.go | 20 ++ pkg/sql/schemachanger/scdecomp/decomp.go | 16 ++ .../scdeps/sctestutils/sctestutils.go | 1 + .../scexec/scmutationexec/BUILD.bazel | 1 + .../scexec/scmutationexec/helpers.go | 15 ++ .../scexec/scmutationexec/policy.go | 57 ++++ .../schemachanger/scop/immediate_mutation.go | 20 ++ .../immediate_mutation_visitor_generated.go | 18 ++ pkg/sql/schemachanger/scpb/elements.proto | 22 +- .../schemachanger/scpb/elements_generated.go | 82 ++++++ pkg/sql/schemachanger/scpb/uml/table.puml | 13 + .../scplan/internal/opgen/BUILD.bazel | 2 + .../scplan/internal/opgen/opgen_policy.go | 33 +++ .../internal/opgen/opgen_policy_name.go | 41 +++ .../scplan/internal/rules/current/BUILD.bazel | 2 + .../rules/current/dep_create_policy.go | 29 ++ .../internal/rules/current/dep_drop_policy.go | 29 ++ .../scplan/internal/rules/current/helpers.go | 8 + .../internal/rules/current/testdata/deprules | 154 ++++++++++- .../scplan/internal/rules/helpers.go | 15 ++ pkg/sql/schemachanger/screl/attr.go | 13 + pkg/sql/schemachanger/screl/attr_string.go | 5 +- pkg/sql/schemachanger/screl/scalars.go | 2 +- pkg/sql/schemachanger/sctest/end_to_end.go | 1 + pkg/sql/schemachanger/sctest/framework.go | 3 + .../schemachanger/sctest_generated_test.go | 84 ++++++ .../create_policy/create_policy.definition | 7 + .../create_policy/create_policy.explain | 29 ++ .../create_policy/create_policy.explain_shape | 8 + .../create_policy/create_policy.side_effects | 55 ++++ .../drop_policy/drop_policy.definition | 10 + .../drop_policy/drop_policy.explain | 32 +++ .../drop_policy/drop_policy.explain_shape | 11 + .../drop_policy/drop_policy.side_effects | 52 ++++ pkg/sql/sem/catid/ids.go | 6 + pkg/sql/sem/tree/alter_policy.go | 4 +- pkg/sql/sem/tree/create_policy.go | 4 +- pkg/sql/sqlerrors/errors.go | 6 + .../lint/passes/redactcheck/redactcheck.go | 1 + 61 files changed, 1474 insertions(+), 178 deletions(-) create mode 100644 pkg/sql/schemachanger/scexec/scmutationexec/policy.go create mode 100644 pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy.go create mode 100644 pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy_name.go create mode 100644 pkg/sql/schemachanger/scplan/internal/rules/current/dep_create_policy.go create mode 100644 pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_policy.go create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.definition create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain_shape create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.definition create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain_shape create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects diff --git a/pkg/ccl/schemachangerccl/backup_base_generated_test.go b/pkg/ccl/schemachangerccl/backup_base_generated_test.go index 5ac6ca1ff335..79c229ebd79b 100644 --- a/pkg/ccl/schemachangerccl/backup_base_generated_test.go +++ b/pkg/ccl/schemachangerccl/backup_base_generated_test.go @@ -435,6 +435,13 @@ func TestBackupRollbacks_base_create_index_create_schema_separate_statements(t * sctest.BackupRollbacks(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupRollbacks_base_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.BackupRollbacks(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupRollbacks_base_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -603,6 +610,13 @@ func TestBackupRollbacks_base_drop_multiple_columns_separate_statements(t *testi sctest.BackupRollbacks(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupRollbacks_base_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.BackupRollbacks(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupRollbacks_base_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1044,6 +1058,13 @@ func TestBackupRollbacksMixedVersion_base_create_index_create_schema_separate_st sctest.BackupRollbacksMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupRollbacksMixedVersion_base_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.BackupRollbacksMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupRollbacksMixedVersion_base_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1212,6 +1233,13 @@ func TestBackupRollbacksMixedVersion_base_drop_multiple_columns_separate_stateme sctest.BackupRollbacksMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupRollbacksMixedVersion_base_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.BackupRollbacksMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupRollbacksMixedVersion_base_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1653,6 +1681,13 @@ func TestBackupSuccess_base_create_index_create_schema_separate_statements(t *te sctest.BackupSuccess(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupSuccess_base_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.BackupSuccess(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupSuccess_base_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1821,6 +1856,13 @@ func TestBackupSuccess_base_drop_multiple_columns_separate_statements(t *testing sctest.BackupSuccess(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupSuccess_base_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.BackupSuccess(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupSuccess_base_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -2262,6 +2304,13 @@ func TestBackupSuccessMixedVersion_base_create_index_create_schema_separate_stat sctest.BackupSuccessMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupSuccessMixedVersion_base_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.BackupSuccessMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupSuccessMixedVersion_base_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -2430,6 +2479,13 @@ func TestBackupSuccessMixedVersion_base_drop_multiple_columns_separate_statement sctest.BackupSuccessMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestBackupSuccessMixedVersion_base_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.BackupSuccessMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestBackupSuccessMixedVersion_base_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/cli/testdata/doctor/test_recreate_zipdir b/pkg/cli/testdata/doctor/test_recreate_zipdir index 116cc9659648..1a955a7f541f 100644 --- a/pkg/cli/testdata/doctor/test_recreate_zipdir +++ b/pkg/cli/testdata/doctor/test_recreate_zipdir @@ -14,5 +14,5 @@ SELECT crdb_internal.unsafe_upsert_namespace_entry(100, 0, 'public', 101, true); SELECT crdb_internal.unsafe_upsert_descriptor(102, decode('125a0a08706f73746772657310661a300a0b0a0561646d696e100218020a0d0a067075626c696310801018000a0a0a04726f6f74100218021204726f6f741803220028013a0c0a067075626c69631202086740004a005a0210007000', 'hex'), true); SELECT crdb_internal.unsafe_upsert_namespace_entry(0, 0, 'postgres', 102, true); SELECT crdb_internal.unsafe_upsert_descriptor(103, decode('2249086612067075626c6963186722310a0b0a0561646d696e100218020a0d0a067075626c696310840418000a0a0a04726f6f7410021802120561646d696e18032a00300140004a007000', 'hex'), true); -SELECT crdb_internal.unsafe_upsert_descriptor(104, decode('0af7020a01741868206428013a0042260a016910011a0c08011040180030005014600020013000680070007800800100880100980100423a0a05726f77696410021a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480352720a06745f706b6579100118012205726f7769642a0169300240004a10080010001a00200028003000380040005a0070017a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c801d88aed86ddb7c5e517d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741803800101880103980100b2011b0a077072696d61727910001a01691a05726f776964200120022801b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c00265c80200e00200800300880302a80300b00300d00300d80300e00300f80300880400', 'hex'), true); +SELECT crdb_internal.unsafe_upsert_descriptor(104, decode('0afa020a01741868206428013a0042260a016910011a0c08011040180030005014600020013000680070007800800100880100980100423a0a05726f77696410021a0c08011040180030005014600020002a0e756e697175655f726f77696428293001680070007800800100880100980100480352720a06745f706b6579100118012205726f7769642a0169300240004a10080010001a00200028003000380040005a0070017a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c801d88aed86ddb7c5e517d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741803800101880103980100b2011b0a077072696d61727910001a01691a05726f776964200120022801b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c00265c80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400', 'hex'), true); COMMIT; diff --git a/pkg/cli/testdata/doctor/test_recreate_zipdir-json b/pkg/cli/testdata/doctor/test_recreate_zipdir-json index eced69a7788c..225551ff8553 100644 --- a/pkg/cli/testdata/doctor/test_recreate_zipdir-json +++ b/pkg/cli/testdata/doctor/test_recreate_zipdir-json @@ -19,16 +19,16 @@ SELECT crdb_internal.unsafe_upsert_descriptor(104, decode('12560a046d6f767210681 SELECT crdb_internal.unsafe_upsert_namespace_entry(0, 0, 'movr', 104, true); SELECT crdb_internal.unsafe_upsert_descriptor(105, decode('2249086812067075626c6963186922310a0b0a0561646d696e100218020a0d0a067075626c696310840418000a0a0a04726f6f7410021802120561646d696e18022a00300140004a007000', 'hex'), true); SELECT crdb_internal.unsafe_upsert_namespace_entry(104, 0, 'public', 105, true); -SELECT crdb_internal.unsafe_upsert_descriptor(106, decode('0aec050a057573657273186a206828013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422a0a046369747910021a0d0807100018003007509308600020003000680070007800800100880100980100422a0a046e616d6510031a0d0807100018003007509308600020013000680070007800800100880100980100422d0a076164647265737310041a0d080710001800300750930860002001300068007000780080010088010098010042310a0b6372656469745f6361726410051a0d08071000180030075093086000200130006800700078008001008801009801004806529a010a0a75736572735f706b657910011801220463697479220269642a046e616d652a07616464726573732a0b6372656469745f6361726430023001400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c801f8c2998282bfb0b317d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b2013d0a077072696d61727910001a0269641a04636974791a046e616d651a07616464726573731a0b6372656469745f63617264200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200aa0233086b1002100418021801206a2a1b76656869636c65735f636974795f6f776e65725f69645f666b657930003800400048007002aa0230086c1002100418021801206a2a1872696465735f636974795f72696465725f69645f666b657930003800400048007003aa023a086f1001100218021801206a2a22757365725f70726f6d6f5f636f6465735f636974795f757365725f69645f666b657930003800400048007002b20200b80200c00269c80200e00200800300880304a80300b00300d00300d80300e00300f80300880400', 'hex'), true); +SELECT crdb_internal.unsafe_upsert_descriptor(106, decode('0aef050a057573657273186a206828013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422a0a046369747910021a0d0807100018003007509308600020003000680070007800800100880100980100422a0a046e616d6510031a0d0807100018003007509308600020013000680070007800800100880100980100422d0a076164647265737310041a0d080710001800300750930860002001300068007000780080010088010098010042310a0b6372656469745f6361726410051a0d08071000180030075093086000200130006800700078008001008801009801004806529a010a0a75736572735f706b657910011801220463697479220269642a046e616d652a07616464726573732a0b6372656469745f6361726430023001400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c801f8c2998282bfb0b317d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b2013d0a077072696d61727910001a0269641a04636974791a046e616d651a07616464726573731a0b6372656469745f63617264200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200aa0233086b1002100418021801206a2a1b76656869636c65735f636974795f6f776e65725f69645f666b657930003800400048007002aa0230086c1002100418021801206a2a1872696465735f636974795f72696465725f69645f666b657930003800400048007003aa023a086f1001100218021801206a2a22757365725f70726f6d6f5f636f6465735f636974795f757365725f69645f666b657930003800400048007002b20200b80200c00269c80200e00200800300880304a80300b00300d00300d80300e00300f80300880400980400', 'hex'), true); SELECT crdb_internal.unsafe_upsert_namespace_entry(104, 105, 'users', 106, true); -SELECT crdb_internal.unsafe_upsert_descriptor(107, decode('0ac1080a0876656869636c6573186b206828013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422a0a046369747910021a0d0807100018003007509308600020003000680070007800800100880100980100422a0a047479706510031a0d0807100018003007509308600020013000680070007800800100880100980100422e0a086f776e65725f696410041a0d080e10001800300050861760002001300068007000780080010088010098010042330a0d6372656174696f6e5f74696d6510051a0d080510001800300050da08600020013000680070007800800100880100980100422c0a0673746174757310061a0d080710001800300750930860002001300068007000780080010088010098010042360a1063757272656e745f6c6f636174696f6e10071a0d080710001800300750930860002001300068007000780080010088010098010042290a0365787410081a0d081210001800300050da1d600020013000680070007800800100880100980100480952c5010a0d76656869636c65735f706b657910011801220463697479220269642a04747970652a086f776e65725f69642a0d6372656174696f6e5f74696d652a067374617475732a1063757272656e745f6c6f636174696f6e2a0365787430023001400040004a10080010001a00200028003000380040005a007003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c801a8c1b58582bfb0b317d00101e00100e90100000000000000005a9b010a2576656869636c65735f6175746f5f696e6465785f666b5f636974795f7265665f75736572731002180022046369747922086f776e65725f6964300230043801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c801a8c1b58582bfb0b317d00100e00100e901000000000000000060036a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b201650a077072696d61727910001a0269641a04636974791a04747970651a086f776e65725f69641a0d6372656174696f6e5f74696d651a067374617475731a1063757272656e745f6c6f636174696f6e1a03657874200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200a20233086b1002100418021801206a2a1b76656869636c65735f636974795f6f776e65725f69645f666b657930003800400048007002aa023a086c1003100518021801206b2a2272696465735f76656869636c655f636974795f76656869636c655f69645f666b657930003800400048007004b20200b80200c00269c80200e00200800300880305a80300b00300d00300d80300e00300f80300880400', 'hex'), true); +SELECT crdb_internal.unsafe_upsert_descriptor(107, decode('0ac4080a0876656869636c6573186b206828013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422a0a046369747910021a0d0807100018003007509308600020003000680070007800800100880100980100422a0a047479706510031a0d0807100018003007509308600020013000680070007800800100880100980100422e0a086f776e65725f696410041a0d080e10001800300050861760002001300068007000780080010088010098010042330a0d6372656174696f6e5f74696d6510051a0d080510001800300050da08600020013000680070007800800100880100980100422c0a0673746174757310061a0d080710001800300750930860002001300068007000780080010088010098010042360a1063757272656e745f6c6f636174696f6e10071a0d080710001800300750930860002001300068007000780080010088010098010042290a0365787410081a0d081210001800300050da1d600020013000680070007800800100880100980100480952c5010a0d76656869636c65735f706b657910011801220463697479220269642a04747970652a086f776e65725f69642a0d6372656174696f6e5f74696d652a067374617475732a1063757272656e745f6c6f636174696f6e2a0365787430023001400040004a10080010001a00200028003000380040005a007003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c801a8c1b58582bfb0b317d00101e00100e90100000000000000005a9b010a2576656869636c65735f6175746f5f696e6465785f666b5f636974795f7265665f75736572731002180022046369747922086f776e65725f6964300230043801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c801a8c1b58582bfb0b317d00100e00100e901000000000000000060036a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b201650a077072696d61727910001a0269641a04636974791a04747970651a086f776e65725f69641a0d6372656174696f6e5f74696d651a067374617475731a1063757272656e745f6c6f636174696f6e1a03657874200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200a20233086b1002100418021801206a2a1b76656869636c65735f636974795f6f776e65725f69645f666b657930003800400048007002aa023a086c1003100518021801206b2a2272696465735f76656869636c655f636974795f76656869636c655f69645f666b657930003800400048007004b20200b80200c00269c80200e00200800300880305a80300b00300d00300d80300e00300f80300880400980400', 'hex'), true); SELECT crdb_internal.unsafe_upsert_namespace_entry(104, 105, 'vehicles', 107, true); -SELECT crdb_internal.unsafe_upsert_descriptor(108, decode('0aa30c0a057269646573186c206828013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422a0a046369747910021a0d080710001800300750930860002000300068007000780080010088010098010042320a0c76656869636c655f6369747910031a0d0807100018003007509308600020013000680070007800800100880100980100422e0a0872696465725f696410041a0d080e10001800300050861760002001300068007000780080010088010098010042300a0a76656869636c655f696410051a0d080e10001800300050861760002001300068007000780080010088010098010042330a0d73746172745f6164647265737310061a0d080710001800300750930860002001300068007000780080010088010098010042310a0b656e645f6164647265737310071a0d080710001800300750930860002001300068007000780080010088010098010042300a0a73746172745f74696d6510081a0d080510001800300050da08600020013000680070007800800100880100980100422e0a08656e645f74696d6510091a0d080510001800300050da08600020013000680070007800800100880100980100422d0a07726576656e7565100a1a0d08031002180a300050a40d600020013000680070007800800100880100980100480b52e7010a0a72696465735f706b657910011801220463697479220269642a0c76656869636c655f636974792a0872696465725f69642a0a76656869636c655f69642a0d73746172745f616464726573732a0b656e645f616464726573732a0a73746172745f74696d652a08656e645f74696d652a07726576656e756530023001400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80188f68d8982bfb0b317d00101e00100e90100000000000000005a98010a2272696465735f6175746f5f696e6465785f666b5f636974795f7265665f757365727310021800220463697479220872696465725f6964300230043801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80188f68d8982bfb0b317d00100e00100e90100000000000000005aaf010a2d72696465735f6175746f5f696e6465785f666b5f76656869636c655f636974795f7265665f76656869636c657310031800220c76656869636c655f63697479220a76656869636c655f69643003300538023801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80188f68d8982bfb0b317d00100e00100e901000000000000000060046a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100a2013a0a1376656869636c655f63697479203d20636974791217636865636b5f76656869636c655f636974795f63697479180128022803300038004002b2018a010a077072696d61727910001a0269641a04636974791a0c76656869636c655f636974791a0872696465725f69641a0a76656869636c655f69641a0d73746172745f616464726573731a0b656e645f616464726573731a0a73746172745f74696d651a08656e645f74696d651a07726576656e7565200120022003200420052006200720082009200a2800b80101c20100e80100f2010408001200f801008002009202009a0200a20230086c1002100418021801206a2a1872696465735f636974795f72696465725f69645f666b657930003800400048007003a2023a086c1003100518021801206b2a2272696465735f76656869636c655f636974795f76656869636c655f69645f666b657930003800400048007004aa0244086d1001100218021801206c2a2c76656869636c655f6c6f636174696f6e5f686973746f726965735f636974795f726964655f69645f666b657930003800400048007002b20200b80200c00269c80200e00200800300880305a80300b00300d00300d80300e00300f80300880400', 'hex'), true); +SELECT crdb_internal.unsafe_upsert_descriptor(108, decode('0aa60c0a057269646573186c206828013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422a0a046369747910021a0d080710001800300750930860002000300068007000780080010088010098010042320a0c76656869636c655f6369747910031a0d0807100018003007509308600020013000680070007800800100880100980100422e0a0872696465725f696410041a0d080e10001800300050861760002001300068007000780080010088010098010042300a0a76656869636c655f696410051a0d080e10001800300050861760002001300068007000780080010088010098010042330a0d73746172745f6164647265737310061a0d080710001800300750930860002001300068007000780080010088010098010042310a0b656e645f6164647265737310071a0d080710001800300750930860002001300068007000780080010088010098010042300a0a73746172745f74696d6510081a0d080510001800300050da08600020013000680070007800800100880100980100422e0a08656e645f74696d6510091a0d080510001800300050da08600020013000680070007800800100880100980100422d0a07726576656e7565100a1a0d08031002180a300050a40d600020013000680070007800800100880100980100480b52e7010a0a72696465735f706b657910011801220463697479220269642a0c76656869636c655f636974792a0872696465725f69642a0a76656869636c655f69642a0d73746172745f616464726573732a0b656e645f616464726573732a0a73746172745f74696d652a08656e645f74696d652a07726576656e756530023001400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80188f68d8982bfb0b317d00101e00100e90100000000000000005a98010a2272696465735f6175746f5f696e6465785f666b5f636974795f7265665f757365727310021800220463697479220872696465725f6964300230043801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80188f68d8982bfb0b317d00100e00100e90100000000000000005aaf010a2d72696465735f6175746f5f696e6465785f666b5f76656869636c655f636974795f7265665f76656869636c657310031800220c76656869636c655f63697479220a76656869636c655f69643003300538023801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80188f68d8982bfb0b317d00100e00100e901000000000000000060046a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100a2013a0a1376656869636c655f63697479203d20636974791217636865636b5f76656869636c655f636974795f63697479180128022803300038004002b2018a010a077072696d61727910001a0269641a04636974791a0c76656869636c655f636974791a0872696465725f69641a0a76656869636c655f69641a0d73746172745f616464726573731a0b656e645f616464726573731a0a73746172745f74696d651a08656e645f74696d651a07726576656e7565200120022003200420052006200720082009200a2800b80101c20100e80100f2010408001200f801008002009202009a0200a20230086c1002100418021801206a2a1872696465735f636974795f72696465725f69645f666b657930003800400048007003a2023a086c1003100518021801206b2a2272696465735f76656869636c655f636974795f76656869636c655f69645f666b657930003800400048007004aa0244086d1001100218021801206c2a2c76656869636c655f6c6f636174696f6e5f686973746f726965735f636974795f726964655f69645f666b657930003800400048007002b20200b80200c00269c80200e00200800300880305a80300b00300d00300d80300e00300f80300880400980400', 'hex'), true); SELECT crdb_internal.unsafe_upsert_namespace_entry(104, 105, 'rides', 108, true); -SELECT crdb_internal.unsafe_upsert_descriptor(109, decode('0ab6050a1a76656869636c655f6c6f636174696f6e5f686973746f72696573186d206828013a00422a0a046369747910011a0d0807100018003007509308600020003000680070007800800100880100980100422d0a07726964655f696410021a0d080e100018003000508617600020003000680070007800800100880100980100422f0a0974696d657374616d7010031a0d080510001800300050da0860002000300068007000780080010088010098010042290a036c617410041a0d080210401800300050bd05600020013000680070007800800100880100980100422a0a046c6f6e6710051a0d080210401800300050bd05600020013000680070007800800100880100980100480652b0010a1f76656869636c655f6c6f636174696f6e5f686973746f726965735f706b6579100118012204636974792207726964655f6964220974696d657374616d702a036c61742a046c6f6e673001300230034000400040004a10080010001a00200028003000380040005a00700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c8018094eb8c82bfb0b317d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b2013c0a077072696d61727910001a04636974791a07726964655f69641a0974696d657374616d701a036c61741a046c6f6e67200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200a20244086d1001100218021801206c2a2c76656869636c655f6c6f636174696f6e5f686973746f726965735f636974795f726964655f69645f666b657930003800400048007002b20200b80200c00269c80200e00200800300880303a80300b00300d00300d80300e00300f80300880400', 'hex'), true); +SELECT crdb_internal.unsafe_upsert_descriptor(109, decode('0ab9050a1a76656869636c655f6c6f636174696f6e5f686973746f72696573186d206828013a00422a0a046369747910011a0d0807100018003007509308600020003000680070007800800100880100980100422d0a07726964655f696410021a0d080e100018003000508617600020003000680070007800800100880100980100422f0a0974696d657374616d7010031a0d080510001800300050da0860002000300068007000780080010088010098010042290a036c617410041a0d080210401800300050bd05600020013000680070007800800100880100980100422a0a046c6f6e6710051a0d080210401800300050bd05600020013000680070007800800100880100980100480652b0010a1f76656869636c655f6c6f636174696f6e5f686973746f726965735f706b6579100118012204636974792207726964655f6964220974696d657374616d702a036c61742a046c6f6e673001300230034000400040004a10080010001a00200028003000380040005a00700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c8018094eb8c82bfb0b317d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b2013c0a077072696d61727910001a04636974791a07726964655f69641a0974696d657374616d701a036c61741a046c6f6e67200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200a20244086d1001100218021801206c2a2c76656869636c655f6c6f636174696f6e5f686973746f726965735f636974795f726964655f69645f666b657930003800400048007002b20200b80200c00269c80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400', 'hex'), true); SELECT crdb_internal.unsafe_upsert_namespace_entry(104, 105, 'vehicle_location_histories', 109, true); -SELECT crdb_internal.unsafe_upsert_descriptor(110, decode('0a8c050a0b70726f6d6f5f636f646573186e206828013a00422a0a04636f646510011a0d080710001800300750930860002000300068007000780080010088010098010042310a0b6465736372697074696f6e10021a0d080710001800300750930860002001300068007000780080010088010098010042330a0d6372656174696f6e5f74696d6510031a0d080510001800300050da0860002001300068007000780080010088010098010042350a0f65787069726174696f6e5f74696d6510041a0d080510001800300050da08600020013000680070007800800100880100980100422b0a0572756c657310051a0d081210001800300050da1d600020013000680070007800800100880100980100480652b2010a1070726f6d6f5f636f6465735f706b6579100118012204636f64652a0b6465736372697074696f6e2a0d6372656174696f6e5f74696d652a0f65787069726174696f6e5f74696d652a0572756c6573300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c801a8df8a9082bfb0b317d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b201510a077072696d61727910001a04636f64651a0b6465736372697074696f6e1a0d6372656174696f6e5f74696d651a0f65787069726174696f6e5f74696d651a0572756c6573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c00269c80200e00200800300880302a80300b00300d00300d80300e00300f80300880400', 'hex'), true); +SELECT crdb_internal.unsafe_upsert_descriptor(110, decode('0a8f050a0b70726f6d6f5f636f646573186e206828013a00422a0a04636f646510011a0d080710001800300750930860002000300068007000780080010088010098010042310a0b6465736372697074696f6e10021a0d080710001800300750930860002001300068007000780080010088010098010042330a0d6372656174696f6e5f74696d6510031a0d080510001800300050da0860002001300068007000780080010088010098010042350a0f65787069726174696f6e5f74696d6510041a0d080510001800300050da08600020013000680070007800800100880100980100422b0a0572756c657310051a0d081210001800300050da1d600020013000680070007800800100880100980100480652b2010a1070726f6d6f5f636f6465735f706b6579100118012204636f64652a0b6465736372697074696f6e2a0d6372656174696f6e5f74696d652a0f65787069726174696f6e5f74696d652a0572756c6573300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c801a8df8a9082bfb0b317d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b201510a077072696d61727910001a04636f64651a0b6465736372697074696f6e1a0d6372656174696f6e5f74696d651a0f65787069726174696f6e5f74696d651a0572756c6573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c00269c80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400', 'hex'), true); SELECT crdb_internal.unsafe_upsert_namespace_entry(104, 105, 'promo_codes', 110, true); -SELECT crdb_internal.unsafe_upsert_descriptor(111, decode('0aaf050a10757365725f70726f6d6f5f636f646573186f206828013a00422a0a046369747910011a0d0807100018003007509308600020003000680070007800800100880100980100422d0a07757365725f696410021a0d080e100018003000508617600020003000680070007800800100880100980100422a0a04636f646510031a0d0807100018003007509308600020003000680070007800800100880100980100422f0a0974696d657374616d7010041a0d080510001800300050da0860002001300068007000780080010088010098010042300a0b75736167655f636f756e7410051a0c08011040180030005014600020013000680070007800800100880100980100480652ae010a15757365725f70726f6d6f5f636f6465735f706b6579100118012204636974792207757365725f69642204636f64652a0974696d657374616d702a0b75736167655f636f756e743001300230034000400040004a10080010001a00200028003000380040005a00700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80190d1b29382bfb0b317d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b201440a077072696d61727910001a04636974791a07757365725f69641a04636f64651a0974696d657374616d701a0b75736167655f636f756e74200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200a2023a086f1001100218021801206a2a22757365725f70726f6d6f5f636f6465735f636974795f757365725f69645f666b657930003800400048007002b20200b80200c00269c80200e00200800300880303a80300b00300d00300d80300e00300f80300880400', 'hex'), true); +SELECT crdb_internal.unsafe_upsert_descriptor(111, decode('0ab2050a10757365725f70726f6d6f5f636f646573186f206828013a00422a0a046369747910011a0d0807100018003007509308600020003000680070007800800100880100980100422d0a07757365725f696410021a0d080e100018003000508617600020003000680070007800800100880100980100422a0a04636f646510031a0d0807100018003007509308600020003000680070007800800100880100980100422f0a0974696d657374616d7010041a0d080510001800300050da0860002001300068007000780080010088010098010042300a0b75736167655f636f756e7410051a0c08011040180030005014600020013000680070007800800100880100980100480652ae010a15757365725f70726f6d6f5f636f6465735f706b6579100118012204636974792207757365725f69642204636f64652a0974696d657374616d702a0b75736167655f636f756e743001300230034000400040004a10080010001a00200028003000380040005a00700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80190d1b29382bfb0b317d00101e00100e901000000000000000060026a210a0b0a0561646d696e100218020a0a0a04726f6f74100218021204726f6f741802800101880103980100b201440a077072696d61727910001a04636974791a07757365725f69641a04636f64651a0974696d657374616d701a0b75736167655f636f756e74200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200a2023a086f1001100218021801206a2a22757365725f70726f6d6f5f636f6465735f636974795f757365725f69645f666b657930003800400048007002b20200b80200c00269c80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400', 'hex'), true); SELECT crdb_internal.unsafe_upsert_namespace_entry(104, 105, 'user_promo_codes', 111, true); COMMIT; diff --git a/pkg/sql/catalog/bootstrap/testdata/testdata b/pkg/sql/catalog/bootstrap/testdata/testdata index 9a007160ec8f..bb12fbf22e8d 100644 --- a/pkg/sql/catalog/bootstrap/testdata/testdata +++ b/pkg/sql/catalog/bootstrap/testdata/testdata @@ -1,69 +1,69 @@ -system hash=8e0584706f674381cb24315b2254e0e8ba091a6246c670d4243fe9603ad4cb5e +system hash=6005356d71ec3b4c026faa14e17f8f1ba9952de6aaa0f6d58e8e773de846cc1b ---- [{"key":"8b"} ,{"key":"8b89898a89","value":"0312470a0673797374656d10011a250a0d0a0561646d696e1080101880100a0c0a04726f6f7410801018801012046e6f646518032200280140004a006a0a08d8843d1003180020127000"} -,{"key":"8b898b8a89","value":"030a94030a0a64657363726970746f721803200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a64657363726970746f7210021a0c08081000180030005011600020013000680070007800800100880100980100480352710a077072696d61727910011801220269642a0a64657363726970746f72300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201130a077072696d61727910001a02696420012800b201240a1066616d5f325f64657363726970746f7210021a0a64657363726970746f7220022802b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b898c8a89","value":"030acd050a0575736572731804200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042330a0e68617368656450617373776f726410021a0c0808100018003000501160002001300068007000780080010088010098010042320a066973526f6c6510031a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055290010a077072696d617279100118012208757365726e616d652a0e68617368656450617373776f72642a066973526f6c652a07757365725f6964300140004a10080010001a00200028003000380040005a007002700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a740a1175736572735f757365725f69645f696478100218012207757365725f69643004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201240a077072696d61727910001a08757365726e616d651a07757365725f6964200120042804b2012c0a1466616d5f325f68617368656450617373776f726410021a0e68617368656450617373776f726420022802b2011c0a0c66616d5f335f6973526f6c6510031a066973526f6c6520032803b80104c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b898d8a89","value":"030a83030a057a6f6e65731805200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422b0a06636f6e66696710021a0c080810001800300050116000200130006800700078008001008801009801004803526d0a077072696d61727910011801220269642a06636f6e666967300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201130a077072696d61727910001a02696420012800b2011c0a0c66616d5f325f636f6e66696710021a06636f6e66696720022802b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b898e8a89","value":"030ac2040a0873657474696e67731806200128013a0042290a046e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510021a0c0807100018003000501960002000300068007000780080010088010098010042440a0b6c6173745570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422e0a0976616c75655479706510041a0c080710001800300050196000200130006800700078008001008801009801004805528a010a077072696d6172791001180122046e616d652a0576616c75652a0b6c617374557064617465642a0976616c756554797065300140004a10080010001a00200028003000380040005a007002700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201590a2666616d5f305f6e616d655f76616c75655f6c617374557064617465645f76616c75655479706510001a046e616d651a0576616c75651a0b6c617374557064617465641a0976616c75655479706520012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b898f8a89","value":"030adf020a1164657363726970746f725f69645f7365711807200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011e0801100118ffffffffffffffff7f20012800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89908a89","value":"030afe060a0774656e616e74731808200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042310a0661637469766510021a0c08001000180030005010600020002a0474727565300168007000780080010088010098010042290a04696e666f10031a0c0808100018003000501160002001300068007000780080010088010098010042290a046e616d6510041a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a646174615f737461746510051a0c0801104018003000501460002001300068007000780080010088010098010042310a0c736572766963655f6d6f646510061a0c080110401800300050146000200130006800700078008001008801009801004807529b010a077072696d61727910011801220269642a066163746976652a04696e666f2a046e616d652a0a646174615f73746174652a0c736572766963655f6d6f6465300140004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a700a1074656e616e74735f6e616d655f6964781002180122046e616d653004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a80010a1874656e616e74735f736572766963655f6d6f64655f69647810031800220c736572766963655f6d6f64653006380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060046a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2014b0a077072696d61727910001a0269641a066163746976651a04696e666f1a046e616d651a0a646174615f73746174651a0c736572766963655f6d6f64652001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89918a89","value":"030aba030a0f726567696f6e5f6c6976656e6573731809200128013a0042300a0b637264625f726567696f6e10011a0c0808100018003000501160002000300068007000780080010088010098010042340a0e756e617661696c61626c655f617410021a0d080510001800300050da086000200130006800700078008001008801009801004803528b010a14726567696f6e5f6c6976656e6573735f706b657910011801220b637264625f726567696f6e2a0e756e617661696c61626c655f6174300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012e0a077072696d61727910001a0b637264625f726567696f6e1a0e756e617661696c61626c655f6174200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89938a89","value":"030a80050a056c65617365180b200128013a00422c0a07646573635f696410011a0c08011040180030005014600020003000680070007800800100880100980100422c0a0776657273696f6e10021a0c0801104018003000501460002000300068007000780080010088010098010042340a0f73716c5f696e7374616e63655f696410031a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a73657373696f6e5f696410041a0c0808100018003000501160002000300068007000780080010088010098010042300a0b637264625f726567696f6e10051a0c08081000180030005011600020003000680070007800800100880100980100480652a9010a077072696d61727910031801220b637264625f726567696f6e2207646573635f6964220776657273696f6e220a73657373696f6e5f69642a0f73716c5f696e7374616e63655f6964300530013002300440004000400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201530a077072696d61727910001a07646573635f69641a0776657273696f6e1a0f73716c5f696e7374616e63655f69641a0a73657373696f6e5f69641a0b637264625f726567696f6e200120022003200420052803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89948a89","value":"030a93060a086576656e746c6f67180c200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422e0a096576656e745479706510021a0c08071000180030005019600020003000680070007800800100880100980100422d0a08746172676574494410031a0c0801104018003000501460002000300068007000780080010088010098010042300a0b7265706f7274696e67494410041a0c0801104018003000501460002000300068007000780080010088010098010042290a04696e666f10051a0c0807100018003000501960002001300068007000780080010088010098010042380a08756e69717565494410061a0c08081000180030005011600020002a09757569645f763428293000680070007800800100880100980100480752a8010a077072696d61727910011801220974696d657374616d702208756e6971756549442a096576656e74547970652a0874617267657449442a0b7265706f7274696e6749442a04696e666f30013006400040004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120062800b201220a0f66616d5f325f6576656e745479706510021a096576656e745479706520022802b201200a0e66616d5f335f746172676574494410031a08746172676574494420032803b201260a1166616d5f345f7265706f7274696e67494410041a0b7265706f7274696e67494420042804b201180a0a66616d5f355f696e666f10051a04696e666f20052805b80106c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89958a89","value":"030af2060a0872616e67656c6f67180d200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422c0a0772616e6765494410021a0c08011040180030005014600020003000680070007800800100880100980100422c0a0773746f7265494410031a0c08011040180030005014600020003000680070007800800100880100980100422e0a096576656e745479706510041a0c0807100018003000501960002000300068007000780080010088010098010042310a0c6f7468657252616e6765494410051a0c0801104018003000501460002001300068007000780080010088010098010042290a04696e666f10061a0c08071000180030005019600020013000680070007800800100880100980100423d0a08756e69717565494410071a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100480852b3010a077072696d61727910011801220974696d657374616d702208756e6971756549442a0772616e676549442a0773746f726549442a096576656e74547970652a0c6f7468657252616e676549442a04696e666f30013007400040004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120072800b2011e0a0d66616d5f325f72616e6765494410021a0772616e6765494420022802b2011e0a0d66616d5f335f73746f7265494410031a0773746f7265494420032803b201220a0f66616d5f345f6576656e745479706510041a096576656e745479706520042804b201280a1266616d5f355f6f7468657252616e6765494410051a0c6f7468657252616e6765494420052805b201180a0a66616d5f365f696e666f10061a04696e666f20062806b80107c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89968a89","value":"030aea030a027569180e200128013a0042280a036b657910011a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510021a0c0808100018003000501160002001300068007000780080010088010098010042310a0b6c6173745570646174656410031a0d080510001800300050da086000200030006800700078008001008801009801004804527c0a077072696d6172791001180122036b65792a0576616c75652a0b6c61737455706461746564300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201140a077072696d61727910001a036b657920012800b2011a0a0b66616d5f325f76616c756510021a0576616c756520022802b201260a1166616d5f335f6c6173745570646174656410031a0b6c6173745570646174656420032803b80104c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89978a89","value":"030aac120a046a6f6273180f200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100422b0a0673746174757310021a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f64726f707065645f7061796c6f616410041a0c0808100018003000501160002001300168007000780080010088010098010042350a1064726f707065645f70726f677265737310051a0c0808100018003000501160002001300168007000780080010088010098010042340a0f637265617465645f62795f7479706510061a0c0807100018003000501960002001300068007000780080010088010098010042320a0d637265617465645f62795f696410071a0c0801104018003000501460002001300068007000780080010088010098010042350a10636c61696d5f73657373696f6e5f696410081a0c0808100018003000501160002001300068007000780080010088010098010042360a11636c61696d5f696e7374616e63655f696410091a0c08011040180030005014600020013000680070007800800100880100980100422d0a086e756d5f72756e73100a1a0c08011040180030005014600020013000680070007800800100880100980100422e0a086c6173745f72756e100b1a0d080510001800300050da08600020013000680070007800800100880100980100422d0a086a6f625f74797065100c1a0c08071000180030005019600020013000680070007800800100880100980100422a0a056f776e6572100d1a0c0807100018003000501960002001300068007000780080010088010098010042300a0b6465736372697074696f6e100e1a0c08071000180030005019600020013000680070007800800100880100980100422e0a096572726f725f6d7367100f1a0c08071000180030005019600020013000680070007800800100880100980100422e0a0866696e697368656410101a0d080910001800300050a009600020013000680070007800800100880100980100481152c1020a077072696d61727910011801220269642a067374617475732a07637265617465642a0f64726f707065645f7061796c6f61642a1064726f707065645f70726f67726573732a0f637265617465645f62795f747970652a0d637265617465645f62795f69642a10636c61696d5f73657373696f6e5f69642a11636c61696d5f696e7374616e63655f69642a086e756d5f72756e732a086c6173745f72756e2a086a6f625f747970652a056f776e65722a0b6465736372697074696f6e2a096572726f725f6d73672a0866696e6973686564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b700c700d700e700f70107a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a86010a176a6f62735f7374617475735f637265617465645f696478100218002206737461747573220763726561746564300230033801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aae010a266a6f62735f637265617465645f62795f747970655f637265617465645f62795f69645f69647810031800220f637265617465645f62795f74797065220d637265617465645f62795f69642a06737461747573300630073801400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ac6020a126a6f62735f72756e5f73746174735f696478100418002210636c61696d5f73657373696f6e5f696422067374617475732207637265617465642a086c6173745f72756e2a086e756d5f72756e732a11636c61696d5f696e7374616e63655f696430083002300338014000400040004a10080010001a00200028003000380040005a00700b700a70097a0408002000800100880100900103980100a20106080012001800a80100b20100ba01810173746174757320494e20282772756e6e696e67273a3a3a535452494e472c2027726576657274696e67273a3a3a535452494e472c202770656e64696e67273a3a3a535452494e472c202770617573652d726571756573746564273a3a3a535452494e472c202763616e63656c2d726571756573746564273a3a3a535452494e4729c00100c80100d00100e00100e90100000000000000005a750a116a6f62735f6a6f625f747970655f6964781005180022086a6f625f74797065300c380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201b4010a1f66616d5f305f69645f7374617475735f637265617465645f7061796c6f616410001a0269641a067374617475731a07637265617465641a0f64726f707065645f7061796c6f61641a0f637265617465645f62795f747970651a0d637265617465645f62795f69641a086a6f625f747970651a056f776e65721a0b6465736372697074696f6e1a096572726f725f6d73671a0866696e6973686564200120022003200420062007200c200d200e200f20102800b201220a0870726f677265737310011a1064726f707065645f70726f677265737320052805b2014c0a05636c61696d10021a10636c61696d5f73657373696f6e5f69641a11636c61696d5f696e7374616e63655f69641a086e756d5f72756e731a086c6173745f72756e20082009200a200b2800b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b899b8a89","value":"030a940c0a0c7765625f73657373696f6e731813200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042310a0c68617368656453656372657410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a08757365726e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042420a0963726561746564417410041a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422f0a0965787069726573417410051a0d080510001800300050da08600020003000680070007800800100880100980100422f0a097265766f6b6564417410061a0d080510001800300050da0860002001300068007000780080010088010098010042430a0a6c61737455736564417410071a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422e0a096175646974496e666f10081a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410091a0c080c100018003000501a600020003000680070007800800100880100980100480a52cc010a077072696d61727910011801220269642a0c6861736865645365637265742a08757365726e616d652a096372656174656441742a096578706972657341742a097265766f6b656441742a0a6c6173745573656441742a096175646974496e666f2a07757365725f6964300140004a10080010001a00200028003000380040005a00700270037004700570067007700870097a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f6578706972657341745f6964781002180022096578706972657341743005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f6372656174656441745f6964781003180022096372656174656441743004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f7265766f6b656441745f6964781004180022097265766f6b656441743006380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a81010a1b7765625f73657373696f6e735f6c6173745573656441745f69647810051800220a6c6173745573656441743007380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201c6010a5166616d5f305f69645f6861736865645365637265745f757365726e616d655f6372656174656441745f6578706972657341745f7265766f6b656441745f6c6173745573656441745f6175646974496e666f10001a0269641a0c6861736865645365637265741a08757365726e616d651a096372656174656441741a096578706972657341741a097265766f6b656441741a0a6c6173745573656441741a096175646974496e666f1a07757365725f69642001200220032004200520062007200820092800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b899c8a89","value":"030aaf0a0a107461626c655f737461746973746963731814200128013a00422c0a077461626c65494410011a0c0801104018003000501460002000300068007000780080010088010098010042400a0b737461746973746963494410021a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042290a046e616d6510031a0c08071000180030005019600020013000680070007800800100880100980100423f0a09636f6c756d6e49447310041a1d080f104018003000380150f8075a0c08011040180030005014600060002000300068007000780080010088010098010042420a0963726561746564417410051a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422d0a08726f77436f756e7410061a0c0801104018003000501460002000300068007000780080010088010098010042320a0d64697374696e6374436f756e7410071a0c08011040180030005014600020003000680070007800800100880100980100422e0a096e756c6c436f756e7410081a0c08011040180030005014600020003000680070007800800100880100980100422e0a09686973746f6772616d10091a0c0808100018003000501160002001300068007000780080010088010098010042360a0761766753697a65100a1a0c08011040180030005014600020002a08303a3a3a494e5438300068007000780080010088010098010042350a107061727469616c507265646963617465100b1a0c0807100018003000501960002001300068007000780080010088010098010042340a0f66756c6c5374617469737469634944100c1a0c08011040180030005014600020013000680070007800800100880100980100480d5284020a077072696d6172791001180122077461626c654944220b73746174697374696349442a046e616d652a09636f6c756d6e4944732a096372656174656441742a08726f77436f756e742a0d64697374696e6374436f756e742a096e756c6c436f756e742a09686973746f6772616d2a0761766753697a652a107061727469616c5072656469636174652a0f66756c6c537461746973746963494430013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b20188020a5d66616d5f305f7461626c6549445f73746174697374696349445f6e616d655f636f6c756d6e4944735f6372656174656441745f726f77436f756e745f64697374696e6374436f756e745f6e756c6c436f756e745f686973746f6772616d10001a077461626c6549441a0b73746174697374696349441a046e616d651a09636f6c756d6e4944731a096372656174656441741a08726f77436f756e741a0d64697374696e6374436f756e741a096e756c6c436f756e741a09686973746f6772616d1a0761766753697a651a107061727469616c5072656469636174651a0f66756c6c5374617469737469634944200120022003200420052006200720082009200a200b200c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b899d8a89","value":"030ae3040a096c6f636174696f6e731815200128013a0042300a0b6c6f63616c6974794b657910011a0c0807100018003000501960002000300068007000780080010088010098010042320a0d6c6f63616c69747956616c756510021a0c08071000180030005019600020003000680070007800800100880100980100422e0a086c6174697475646510031a0d0803100f1812300050a40d600020003000680070007800800100880100980100422f0a096c6f6e67697475646510041a0d0803100f1812300050a40d60002000300068007000780080010088010098010048055298010a077072696d61727910011801220b6c6f63616c6974794b6579220d6c6f63616c69747956616c75652a086c617469747564652a096c6f6e67697475646530013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201710a3266616d5f305f6c6f63616c6974794b65795f6c6f63616c69747956616c75655f6c617469747564655f6c6f6e67697475646510001a0b6c6f63616c6974794b65791a0d6c6f63616c69747956616c75651a086c617469747564651a096c6f6e67697475646520012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b899f8a89","value":"030aa20a0a0c726f6c655f6d656d626572731817200128013a0042290a04726f6c6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066d656d62657210021a0c08071000180030005019600020003000680070007800800100880100980100422c0a07697341646d696e10031a0c08001000180030005010600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a600020003000680070007800800100880100980100422e0a096d656d6265725f696410051a0c080c100018003000501a60002000300068007000780080010088010098010048065294010a077072696d617279100118012204726f6c6522066d656d6265722a07697341646d696e2a07726f6c655f69642a096d656d6265725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a750a15726f6c655f6d656d626572735f726f6c655f696478100218002204726f6c653001380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a790a17726f6c655f6d656d626572735f6d656d6265725f6964781003180022066d656d6265723002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7d0a18726f6c655f6d656d626572735f726f6c655f69645f696478100418002207726f6c655f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a81010a1a726f6c655f6d656d626572735f6d656d6265725f69645f6964781005180022096d656d6265725f696430053801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a96010a22726f6c655f6d656d626572735f726f6c655f69645f6d656d6265725f69645f6b6579100618012207726f6c655f696422096d656d6265725f69643004300538013802400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060076a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2011f0a077072696d61727910001a04726f6c651a066d656d626572200120022800b2011e0a0d66616d5f335f697341646d696e10031a07697341646d696e20032803b2011e0a0d66616d5f345f726f6c655f696410041a07726f6c655f696420042804b201220a0f66616d5f355f6d656d6265725f696410051a096d656d6265725f696420052805b80106c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a08a89","value":"030aae040a08636f6d6d656e74731818200128013a0042290a047479706510011a0c08011040180030005014600020003000680070007800800100880100980100422e0a096f626a6563745f696410021a0c08011040180030005014600020003000680070007800800100880100980100422b0a067375625f696410031a0c08011040180030005014600020003000680070007800800100880100980100422c0a07636f6d6d656e7410041a0c080710001800300050196000200030006800700078008001008801009801004805528b010a077072696d6172791001180122047479706522096f626a6563745f696422067375625f69642a07636f6d6d656e743001300230034000400040004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a330a0d0a0561646d696e10e00318e0030a0c0a067075626c6963102018000a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012c0a077072696d61727910001a04747970651a096f626a6563745f69641a067375625f69642001200220032800b2011e0a0d66616d5f345f636f6d6d656e7410041a07636f6d6d656e7420042804b80105c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a18a89","value":"030aa9060a1c7265706c69636174696f6e5f636f6e73747261696e745f73746174731819200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c0801104018003000501460002000300068007000780080010088010098010042290a047479706510031a0c08071000180030005019600020003000680070007800800100880100980100422b0a06636f6e66696710041a0c08071000180030005019600020003000680070007800800100880100980100422e0a097265706f72745f696410051a0c0801104018003000501460002000300068007000780080010088010098010042350a0f76696f6c6174696f6e5f737461727410061a0d080910001800300050a00960002001300068007000780080010088010098010042350a1076696f6c6174696e675f72616e67657310071a0c08011040180030005014600020003000680070007800800100880100980100480852c2010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f69642204747970652206636f6e6669672a097265706f72745f69642a0f76696f6c6174696f6e5f73746172742a1076696f6c6174696e675f72616e676573300130023003300440004000400040004a10080010001a00200028003000380040005a007005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2016c0a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a04747970651a06636f6e6669671a097265706f72745f69641a0f76696f6c6174696f6e5f73746172741a1076696f6c6174696e675f72616e67657320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a28a89","value":"030a92050a1f7265706c69636174696f6e5f637269746963616c5f6c6f63616c6974696573181a200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422d0a086c6f63616c69747910031a0c08071000180030005019600020003000680070007800800100880100980100422e0a097265706f72745f696410041a0c0801104018003000501460002000300068007000780080010088010098010042330a0e61745f7269736b5f72616e67657310051a0c08011040180030005014600020003000680070007800800100880100980100480652a5010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f696422086c6f63616c6974792a097265706f72745f69642a0e61745f7269736b5f72616e6765733001300230034000400040004a10080010001a00200028003000380040005a00700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201510a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a086c6f63616c6974791a097265706f72745f69641a0e61745f7269736b5f72616e676573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a38a89","value":"030a80070a117265706c69636174696f6e5f7374617473181b200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422e0a097265706f72745f696410031a0c0801104018003000501460002000300068007000780080010088010098010042310a0c746f74616c5f72616e67657310041a0c0801104018003000501460002000300068007000780080010088010098010042370a12756e617661696c61626c655f72616e67657310051a0c08011040180030005014600020003000680070007800800100880100980100423c0a17756e6465725f7265706c6963617465645f72616e67657310061a0c08011040180030005014600020003000680070007800800100880100980100423b0a166f7665725f7265706c6963617465645f72616e67657310071a0c08011040180030005014600020003000680070007800800100880100980100480852e0010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f69642a097265706f72745f69642a0c746f74616c5f72616e6765732a12756e617661696c61626c655f72616e6765732a17756e6465725f7265706c6963617465645f72616e6765732a166f7665725f7265706c6963617465645f72616e67657330013002400040004a10080010001a00200028003000380040005a00700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2018e010a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a097265706f72745f69641a0c746f74616c5f72616e6765731a12756e617661696c61626c655f72616e6765731a17756e6465725f7265706c6963617465645f72616e6765731a166f7665725f7265706c6963617465645f72616e67657320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a48a89","value":"030aff020a0c7265706f7274735f6d657461181c200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0967656e65726174656410021a0d080910001800300050a009600020003000680070007800800100880100980100480352700a077072696d61727910011801220269642a0967656e657261746564300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201200a077072696d61727910001a0269641a0967656e657261746564200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a68a89","value":"030a9e040a096e616d657370616365181e200128013a00422d0a08706172656e74494410011a0c0801104018003000501460002000300068007000780080010088010098010042330a0e706172656e74536368656d61494410021a0c0801104018003000501460002000300068007000780080010088010098010042290a046e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042270a02696410041a0c080110401800300050146000200130006800700078008001008801009801004805528d010a077072696d617279100118012208706172656e744944220e706172656e74536368656d61494422046e616d652a0269643001300230034000400040004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201330a077072696d61727910001a08706172656e7449441a0e706172656e74536368656d6149441a046e616d652001200220032800b201140a0866616d5f345f696410041a02696420042804b80105c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a78a89","value":"030aa8050a1170726f7465637465645f74735f6d657461181f200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422c0a0776657273696f6e10021a0c0801104018003000501460002000300068007000780080010088010098010042300a0b6e756d5f7265636f72647310031a0c08011040180030005014600020003000680070007800800100880100980100422e0a096e756d5f7370616e7310041a0c0801104018003000501460002000300068007000780080010088010098010042300a0b746f74616c5f627974657310051a0c08011040180030005014600020003000680070007800800100880100980100480652a0010a077072696d61727910011801220973696e676c65746f6e2a0776657273696f6e2a0b6e756d5f7265636f7264732a096e756d5f7370616e732a0b746f74616c5f6279746573300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201260a0973696e676c65746f6e120f636865636b5f73696e676c65746f6e18002801300038004002b201500a077072696d61727910001a0973696e676c65746f6e1a0776657273696f6e1a0b6e756d5f7265636f7264731a096e756d5f7370616e731a0b746f74616c5f6279746573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a88a89","value":"030a8c060a1470726f7465637465645f74735f7265636f7264731820200128013a0042280a02696410011a0d080e10001800300050861760002000300068007000780080010088010098010042280a02747310021a0d080310001800300050a40d600020003000680070007800800100880100980100422e0a096d6574615f7479706510031a0c0807100018003000501960002000300068007000780080010088010098010042290a046d65746110041a0c08081000180030005011600020013000680070007800800100880100980100422e0a096e756d5f7370616e7310051a0c08011040180030005014600020003000680070007800800100880100980100422a0a057370616e7310061a0c0808100018003000501160002000300068007000780080010088010098010042340a08766572696669656410071a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422b0a0674617267657410081a0c08081000180030005011600020013000680070007800800100880100980100480952aa010a077072696d61727910011801220269642a0274732a096d6574615f747970652a046d6574612a096e756d5f7370616e732a057370616e732a0876657269666965642a06746172676574300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2015a0a077072696d61727910001a0269641a0274731a096d6574615f747970651a046d6574611a096e756d5f7370616e731a057370616e731a0876657269666965641a06746172676574200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a98a89","value":"030a84050a0c726f6c655f6f7074696f6e731821200128013a00422d0a08757365726e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066f7074696f6e10021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055289010a077072696d617279100118012208757365726e616d6522066f7074696f6e2a0576616c75652a07757365725f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a760a1175736572735f757365725f69645f696478100218002207757365725f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201370a077072696d61727910001a08757365726e616d651a066f7074696f6e1a0576616c75651a07757365725f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89aa8a89","value":"030ada030a1773746174656d656e745f62756e646c655f6368756e6b731822200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042300a0b6465736372697074696f6e10021a0c0807100018003000501960002001300068007000780080010088010098010042290a046461746110031a0c080810001800300050116000200030006800700078008001008801009801004804527a0a077072696d61727910011801220269642a0b6465736372697074696f6e2a0464617461300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012a0a077072696d61727910001a0269641a0b6465736372697074696f6e1a04646174612001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ab8a89","value":"030ae50c0a1e73746174656d656e745f646961676e6f73746963735f72657175657374731823200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042350a09636f6d706c6574656410021a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410031a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f646961676e6f73746963735f696410041a0c0801104018003000501460002001300068007000780080010088010098010042320a0c7265717565737465645f617410051a0d080910001800300050a00960002000300068007000780080010088010098010042410a156d696e5f657865637574696f6e5f6c6174656e637910061a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042300a0a657870697265735f617410071a0d080910001800300050a009600020013000680070007800800100880100980100423a0a1473616d706c696e675f70726f626162696c69747910081a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a09706c616e5f6769737410091a0c0807100018003000501960002001300068007000780080010088010098010042330a0e616e74695f706c616e5f67697374100a1a0c0800100018003000501060002001300068007000780080010088010098010042340a087265646163746564100b1a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100480c529f020a077072696d61727910011801220269642a09636f6d706c657465642a1573746174656d656e745f66696e6765727072696e742a1873746174656d656e745f646961676e6f73746963735f69642a0c7265717565737465645f61742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f676973742a087265646163746564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005afb010a0d636f6d706c657465645f696478100218002209636f6d706c65746564220269642a1573746174656d656e745f66696e6765727072696e742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f676973742a08726564616374656430023001400040004a10080010001a00200028003000380040005a0070037006700770087009700a700b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201620a3a73616d706c696e675f70726f626162696c697479204245545745454e20302e303a3a3a464c4f41543820414e4420312e303a3a3a464c4f415438121a636865636b5f73616d706c696e675f70726f626162696c69747918002808300038004002b201cf010a077072696d61727910001a0269641a09636f6d706c657465641a1573746174656d656e745f66696e6765727072696e741a1873746174656d656e745f646961676e6f73746963735f69641a0c7265717565737465645f61741a156d696e5f657865637574696f6e5f6c6174656e63791a0a657870697265735f61741a1473616d706c696e675f70726f626162696c6974791a09706c616e5f676973741a0e616e74695f706c616e5f676973741a087265646163746564200120022003200420052006200720082009200a200b2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ac8a89","value":"030abe060a1573746174656d656e745f646961676e6f73746963731824200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410021a0c08071000180030005019600020003000680070007800800100880100980100422e0a0973746174656d656e7410031a0c0807100018003000501960002000300068007000780080010088010098010042320a0c636f6c6c65637465645f617410041a0d080910001800300050a009600020003000680070007800800100880100980100422b0a05747261636510051a0d081210001800300050da1d60002001300068007000780080010088010098010042430a0d62756e646c655f6368756e6b7310061a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a056572726f7210071a0c08071000180030005019600020013000680070007800800100880100980100480852bc010a077072696d61727910011801220269642a1573746174656d656e745f66696e6765727072696e742a0973746174656d656e742a0c636f6c6c65637465645f61742a0574726163652a0d62756e646c655f6368756e6b732a056572726f72300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2016c0a077072696d61727910001a0269641a1573746174656d656e745f66696e6765727072696e741a0973746174656d656e741a0c636f6c6c65637465645f61741a0574726163651a0d62756e646c655f6368756e6b731a056572726f7220012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ad8a89","value":"030ad3090a0e7363686564756c65645f6a6f62731825200128013a0042400a0b7363686564756c655f696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042320a0d7363686564756c655f6e616d6510021a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210041a0c08071000180030005019600020003000680070007800800100880100980100422e0a086e6578745f72756e10051a0d080910001800300050a00960002001300068007000780080010088010098010042330a0e7363686564756c655f737461746510061a0c0808100018003000501160002001300068007000780080010088010098010042320a0d7363686564756c655f6578707210071a0c0807100018003000501960002001300068007000780080010088010098010042350a107363686564756c655f64657461696c7310081a0c0808100018003000501160002001300068007000780080010088010098010042320a0d6578656375746f725f7479706510091a0c0807100018003000501960002000300068007000780080010088010098010042330a0e657865637574696f6e5f61726773100a1a0c08081000180030005011600020003000680070007800800100880100980100480b52f7010a077072696d61727910011801220b7363686564756c655f69642a0d7363686564756c655f6e616d652a07637265617465642a056f776e65722a086e6578745f72756e2a0e7363686564756c655f73746174652a0d7363686564756c655f657870722a107363686564756c655f64657461696c732a0d6578656375746f725f747970652a0e657865637574696f6e5f61726773300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a700a0c6e6578745f72756e5f6964781002180022086e6578745f72756e3005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201380a05736368656410001a0b7363686564756c655f69641a086e6578745f72756e1a0e7363686564756c655f73746174652001200520062800b201780a056f7468657210011a0d7363686564756c655f6e616d651a07637265617465641a056f776e65721a0d7363686564756c655f657870721a107363686564756c655f64657461696c731a0d6578656375746f725f747970651a0e657865637574696f6e5f61726773200220032004200720082009200a2800b80102c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89af8a89","value":"030aec030a0b73716c6c6976656e6573731827200128013a00422f0a0a73657373696f6e5f696410011a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10021a0d080310001800300050a40d60002000300068007000780080010088010098010042300a0b637264625f726567696f6e10031a0c080810001800300050116000200030006800700078008001008801009801004804528a010a077072696d61727910021801220b637264625f726567696f6e220a73657373696f6e5f69642a0a65787069726174696f6e30033001400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201380a077072696d61727910001a0b637264625f726567696f6e1a0a73657373696f6e5f69641a0a65787069726174696f6e2003200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b08a89","value":"030ad9040a0a6d6967726174696f6e731828200128013a00422a0a056d616a6f7210011a0c08011040180030005014600020003000680070007800800100880100980100422a0a056d696e6f7210021a0c08011040180030005014600020003000680070007800800100880100980100422a0a05706174636810031a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e7465726e616c10041a0c0801104018003000501460002000300068007000780080010088010098010042320a0c636f6d706c657465645f617410051a0d080910001800300050a0096000200030006800700078008001008801009801004806529a010a077072696d6172791001180122056d616a6f7222056d696e6f72220570617463682208696e7465726e616c2a0c636f6d706c657465645f6174300130023003300440004000400040004a10080010001a00200028003000380040005a0070057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201440a077072696d61727910001a056d616a6f721a056d696e6f721a0570617463681a08696e7465726e616c1a0c636f6d706c657465645f6174200120022003200420052805b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b18a89","value":"030ac3030a0b6a6f696e5f746f6b656e731829200128013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422b0a0673656372657410021a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10031a0d080910001800300050a0096000200030006800700078008001008801009801004804527b0a077072696d61727910011801220269642a067365637265742a0a65787069726174696f6e300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012b0a077072696d61727910001a0269641a067365637265741a0a65787069726174696f6e2001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b28a89","value":"030a8f280a1473746174656d656e745f73746174697374696373182a200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410061a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10071a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110081a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310091a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e100a1a0d081210001800300050da1d600020003000680070007800800100880100980100429c020a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38100b1a0c080110201800300050176000200030015a8c016d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f69642c20706c616e5f686173682c207472616e73616374696f6e5f66696e6765727072696e745f696429292c20383a3a3a494e543829680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100c1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d3000680070007800800100880100980100426c0a0d696e64657865735f7573616765100d1a0d081210001800300050da1d6000200130005a3728737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27696e6465786573273a3a3a535452494e4768007000780080010188010098010042710a0f657865637574696f6e5f636f756e74100e1a0c080110401800300050146000200130005a3b2828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27636e74273a3a3a535452494e47293a3a494e5438680070007800800100880100980100428a010a0f736572766963655f6c6174656e6379100f1a0d080210401800300050bd056000200130005a53282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f4154386800700078008001008801009801004297010a0d6370755f73716c5f6e616e6f7310101a0d080210401800300050bd056000200130005a62282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e2763707553514c4e616e6f73273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100429c010a0f636f6e74656e74696f6e5f74696d6510111a0d080210401800300050bd056000200130005a65282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e27636f6e74656e74696f6e54696d65273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f41543868007000780080010088010098010042dc010a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d6510121a0d080210401800300050bd056000200130005a95012828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e3e27636e74273a3a3a535452494e47293a3a464c4f415438202a20282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100428a010a0b7039395f6c6174656e637910131a0d080210401800300050bd056000200130005a57282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e276c6174656e6379496e666f273a3a3a535452494e47292d3e27703939273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100481452c2050a077072696d617279100118012268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e732a0f657865637574696f6e5f636f756e742a0f736572766963655f6c6174656e63792a0d6370755f73716c5f6e616e6f732a0f636f6e74656e74696f6e5f74696d652a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d652a0b7039395f6c6174656e6379300b30013002300330043005300640004000400040004000400040004a10080010001a00200028003000380040005a00700770087009700a700c700e700f70107011701270137a0408002000800100880100900104980101a201c70108011268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f69642209706c616e5f68617368221a7472616e73616374696f6e5f66696e6765727072696e745f6964a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa7010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380b3801380438053806400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a89010a11696e64657865735f75736167655f69647810031800220d696e64657865735f7573616765300d380b38013802380338043805380640004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e00100e90100000000000000005acf010a13657865637574696f6e5f636f756e745f69647810041800220d616767726567617465645f747322086170705f6e616d65220f657865637574696f6e5f636f756e7430013005300e380b38023803380438064000400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ad1010a13736572766963655f6c6174656e63795f69647810051800220d616767726567617465645f747322086170705f6e616d65220f736572766963655f6c6174656e637930013005300f380b38023803380438064000400040014a10080010001a00200028003000380040005a00680f7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a116370755f73716c5f6e616e6f735f69647810061800220d616767726567617465645f747322086170705f6e616d65220d6370755f73716c5f6e616e6f73300130053010380b38023803380438064000400040014a10080010001a00200028003000380040005a0068107a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ad1010a13636f6e74656e74696f6e5f74696d655f69647810071800220d616767726567617465645f747322086170705f6e616d65220f636f6e74656e74696f6e5f74696d65300130053011380b38023803380438064000400040014a10080010001a00200028003000380040005a0068117a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005aef010a22746f74616c5f657374696d617465645f657865637574696f6e5f74696d655f69647810081800220d616767726567617465645f747322086170705f6e616d65221e746f74616c5f657374696d617465645f657865637574696f6e5f74696d65300130053012380b38023803380438064000400040014a10080010001a00200028003000380040005a0068127a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac9010a0f7039395f6c6174656e63795f69647810091800220d616767726567617465645f747322086170705f6e616d65220b7039395f6c6174656e6379300130053013380b38023803380438064000400040014a10080010001a00200028003000380040005a0068137a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e9010000000000000000600a6a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201b9020abc01637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e543829126e636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381800280b300038014002b201a4030a077072696d61727910001a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e731a0f657865637574696f6e5f636f756e741a0f736572766963655f6c6174656e63791a0d6370755f73716c5f6e616e6f731a0f636f6e74656e74696f6e5f74696d651a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d651a0b7039395f6c6174656e6379200b200120022003200420052006200720082009200a200c200e200f20102011201220132800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b38a89","value":"030a95200a167472616e73616374696f6e5f73746174697374696373182b200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410041a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10051a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110061a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310071a0d081210001800300050da1d60002000300068007000780080010088010098010042cf010a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3810081a0c080110201800300050176000200030015a656d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f696429292c20383a3a3a494e54382968007000780080010088010098010042710a0f657865637574696f6e5f636f756e7410091a0c080110401800300050146000200130005a3b2828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27636e74273a3a3a535452494e47293a3a494e5438680070007800800100880100980100428a010a0f736572766963655f6c6174656e6379100a1a0d080210401800300050bd056000200130005a53282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f4154386800700078008001008801009801004297010a0d6370755f73716c5f6e616e6f73100b1a0d080210401800300050bd056000200130005a62282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e2763707553514c4e616e6f73273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100429c010a0f636f6e74656e74696f6e5f74696d65100c1a0d080210401800300050bd056000200130005a65282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e27636f6e74656e74696f6e54696d65273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f41543868007000780080010088010098010042dc010a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d65100d1a0d080210401800300050bd056000200130005a95012828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e3e27636e74273a3a3a535452494e47293a3a464c4f415438202a20282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100428a010a0b7039395f6c6174656e6379100e1a0d080210401800300050bd056000200130005a57282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e276c6174656e6379496e666f273a3a3a535452494e47292d3e27703939273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100480f5280040a077072696d617279100118012243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a0f657865637574696f6e5f636f756e742a0f736572766963655f6c6174656e63792a0d6370755f73716c5f6e616e6f732a0f636f6e74656e74696f6e5f74696d652a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d652a0b7039395f6c6174656e637930083001300230033004400040004000400040004a10080010001a00200028003000380040005a007005700670077009700a700b700c700d700e7a0408002000800100880100900104980101a2017b08011243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f6964a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a85010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f69643002380838013803380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005acb010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f747322086170705f6e616d65220f657865637574696f6e5f636f756e743001300330093808380238044000400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a13736572766963655f6c6174656e63795f69647810041800220d616767726567617465645f747322086170705f6e616d65220f736572766963655f6c6174656e637930013003300a3808380238044000400040014a10080010001a00200028003000380040005a00680a7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac9010a116370755f73716c5f6e616e6f735f69647810051800220d616767726567617465645f747322086170705f6e616d65220d6370755f73716c5f6e616e6f7330013003300b3808380238044000400040014a10080010001a00200028003000380040005a00680b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a13636f6e74656e74696f6e5f74696d655f69647810061800220d616767726567617465645f747322086170705f6e616d65220f636f6e74656e74696f6e5f74696d6530013003300c3808380238044000400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005aeb010a22746f74616c5f657374696d617465645f657865637574696f6e5f74696d655f69647810071800220d616767726567617465645f747322086170705f6e616d65221e746f74616c5f657374696d617465645f657865637574696f6e5f74696d6530013003300d3808380238044000400040014a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac5010a0f7039395f6c6174656e63795f69647810081800220d616767726567617465645f747322086170705f6e616d65220b7039395f6c6174656e637930013003300e3808380238044000400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201ef010a9701637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e5438291249636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3818002808300038014002b201b3020a077072696d61727910001a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a0f657865637574696f6e5f636f756e741a0f736572766963655f6c6174656e63791a0d6370755f73716c5f6e616e6f731a0f636f6e74656e74696f6e5f74696d651a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d651a0b7039395f6c6174656e6379200820012002200320042005200620072009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b48a89","value":"030af3050a1664617461626173655f726f6c655f73657474696e6773182c200128013a0042300a0b64617461626173655f696410011a0c080c100018003000501a600020003000680070007800800100880100980100422e0a09726f6c655f6e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100423e0a0873657474696e677310031a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055292010a077072696d61727910011801220b64617461626173655f69642209726f6c655f6e616d652a0873657474696e67732a07726f6c655f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005aae010a2e64617461626173655f726f6c655f73657474696e67735f64617461626173655f69645f726f6c655f69645f6b657910021801220b64617461626173655f69642207726f6c655f69642a0873657474696e6773300130043802400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201400a077072696d61727910001a0b64617461626173655f69641a09726f6c655f6e616d651a0873657474696e67731a07726f6c655f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b58a89","value":"030ac00b0a0c74656e616e745f7573616765182d200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042300a0b696e7374616e63655f696410021a0c0801104018003000501460002000300068007000780080010088010098010042350a106e6578745f696e7374616e63655f696410031a0c0801104018003000501460002000300068007000780080010088010098010042310a0b6c6173745f75706461746510041a0d080510001800300050da0860002000300068007000780080010088010098010042340a0e72755f62757273745f6c696d697410051a0d080210401800300050bd0560002001300068007000780080010088010098010042340a0e72755f726566696c6c5f7261746510061a0d080210401800300050bd0560002001300068007000780080010088010098010042300a0a72755f63757272656e7410071a0d080210401800300050bd0560002001300068007000780080010088010098010042370a1163757272656e745f73686172655f73756d10081a0d080210401800300050bd0560002001300068007000780080010088010098010042360a11746f74616c5f636f6e73756d7074696f6e10091a0c0808100018003000501160002001300068007000780080010088010098010042330a0e696e7374616e63655f6c65617365100a1a0c0808100018003000501160002001300068007000780080010088010098010042310a0c696e7374616e63655f736571100b1a0c0801104018003000501460002001300068007000780080010088010098010042350a0f696e7374616e63655f736861726573100c1a0d080210401800300050bd0560002001300068007000780080010088010098010042320a0d63757272656e745f7261746573100d1a0c08081000180030005011600020013000680070007800800100880100980100422f0a0a6e6578745f7261746573100e1a0c08081000180030005011600020013000680070007800800100880100980100480f52ce020a077072696d61727910011801220974656e616e745f6964220b696e7374616e63655f69642a106e6578745f696e7374616e63655f69642a0b6c6173745f7570646174652a0e72755f62757273745f6c696d69742a0e72755f726566696c6c5f726174652a0a72755f63757272656e742a1163757272656e745f73686172655f73756d2a11746f74616c5f636f6e73756d7074696f6e2a0e696e7374616e63655f6c656173652a0c696e7374616e63655f7365712a0f696e7374616e63655f7368617265732a0d63757272656e745f72617465732a0a6e6578745f726174657330013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c700d700e7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201fc010a077072696d61727910001a0974656e616e745f69641a0b696e7374616e63655f69641a106e6578745f696e7374616e63655f69641a0b6c6173745f7570646174651a0e72755f62757273745f6c696d69741a0e72755f726566696c6c5f726174651a0a72755f63757272656e741a1163757272656e745f73686172655f73756d1a11746f74616c5f636f6e73756d7074696f6e1a0e696e7374616e63655f6c656173651a0c696e7374616e63655f7365711a0f696e7374616e63655f7368617265731a0d63757272656e745f72617465731a0a6e6578745f7261746573200120022003200420052006200720082009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b68a89","value":"030ac8060a0d73716c5f696e7374616e636573182e200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046164647210021a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410031a0c08081000180030005011600020013000680070007800800100880100980100422e0a086c6f63616c69747910041a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0873716c5f6164647210051a0c0807100018003000501960002001300068007000780080010088010098010042300a0b637264625f726567696f6e10061a0c0808100018003000501160002000300068007000780080010088010098010042330a0e62696e6172795f76657273696f6e10071a0c0807100018003000501960002001300068007000780080010088010098010042300a0b69735f647261696e696e6710081a0c08001000180030005010600020013000680070007800800100880100980100480952c3010a077072696d61727910021801220b637264625f726567696f6e220269642a04616464722a0a73657373696f6e5f69642a086c6f63616c6974792a0873716c5f616464722a0e62696e6172795f76657273696f6e2a0b69735f647261696e696e6730063001400040004a10080010001a00200028003000380040005a007002700370047005700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201710a077072696d61727910001a0269641a04616464721a0a73657373696f6e5f69641a086c6f63616c6974791a0873716c5f616464721a0b637264625f726567696f6e1a0e62696e6172795f76657273696f6e1a0b69735f647261696e696e67200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b78a89","value":"030a87040a137370616e5f636f6e66696775726174696f6e73182f200128013a00422e0a0973746172745f6b657910011a0c08081000180030005011600020003000680070007800800100880100980100422c0a07656e645f6b657910021a0c08081000180030005011600020003000680070007800800100880100980100422b0a06636f6e66696710031a0c080810001800300050116000200030006800700078008001008801009801004804527f0a077072696d61727910011801220973746172745f6b65792a07656e645f6b65792a06636f6e666967300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a2012f0a1373746172745f6b6579203c20656e645f6b6579120c636865636b5f626f756e6473180028012802300038004002b2012f0a077072696d61727910001a0973746172745f6b65791a07656e645f6b65791a06636f6e6669672001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b88a89","value":"030ad9020a0b726f6c655f69645f7365711830200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a250a0d0a0561646d696e10a00618a0060a0c0a04726f6f7410a00618a00612046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011a0801106418ffffffff0720642800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ba8a89","value":"030af0050a0f74656e616e745f73657474696e67731832200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c0807100018003000501960002000300068007000780080010088010098010042450a0c6c6173745f7570646174656410041a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422f0a0a76616c75655f7479706510051a0c08071000180030005019600020003000680070007800800100880100980100422b0a06726561736f6e10061a0c08071000180030005019600020013000680070007800800100880100980100480752a5010a077072696d61727910011801220974656e616e745f696422046e616d652a0576616c75652a0c6c6173745f757064617465642a0a76616c75655f747970652a06726561736f6e30013002400040004a10080010001a00200028003000380040005a0070037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b20185010a3966616d5f305f74656e616e745f69645f6e616d655f76616c75655f6c6173745f757064617465645f76616c75655f747970655f726561736f6e10001a0974656e616e745f69641a046e616d651a0576616c75651a0c6c6173745f757064617465641a0a76616c75655f747970651a06726561736f6e2001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89bb8a89","value":"030abe030a0a7370616e5f636f756e741833200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422f0a0a7370616e5f636f756e7410021a0c08011040180030005014600020003000680070007800800100880100980100480352780a077072696d61727910011801220973696e676c65746f6e2a0a7370616e5f636f756e74300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201210a0973696e676c65746f6e120a73696e676c655f726f7718002801300038004002b201280a077072696d61727910001a0973696e676c65746f6e1a0a7370616e5f636f756e74200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89bc8a89","value":"030adf070a0a70726976696c656765731834200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042290a047061746810021a0c0807100018003000501960002000300068007000780080010088010098010042400a0a70726976696c6567657310031a1d080f100018003000380750f1075a0c08071000180030005019600060002000300068007000780080010088010098010042430a0d6772616e745f6f7074696f6e7310041a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07757365725f696410051a0c080c100018003000501a6000200030006800700078008001008801009801004806529d010a077072696d617279100118012208757365726e616d652204706174682a0a70726976696c656765732a0d6772616e745f6f7074696f6e732a07757365725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00103e00100e90100000000000000005aa7010a1b70726976696c656765735f706174685f757365725f69645f6b6579100218012204706174682207757365725f69642a0a70726976696c656765732a0d6772616e745f6f7074696f6e73300230053801400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa7010a1c70726976696c656765735f706174685f757365726e616d655f6b6579100318012204706174682208757365726e616d652a0a70726976696c656765732a0d6772616e745f6f7074696f6e7330023001400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2014b0a077072696d61727910001a08757365726e616d651a04706174681a0a70726976696c656765731a0d6772616e745f6f7074696f6e731a07757365725f6964200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880304a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89bd8a89","value":"030ada060a1465787465726e616c5f636f6e6e656374696f6e731835200128013a0042340a0f636f6e6e656374696f6e5f6e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042400a077570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f636f6e6e656374696f6e5f7479706510041a0c0807100018003000501960002000300068007000780080010088010098010042370a12636f6e6e656374696f6e5f64657461696c7310051a0c08081000180030005011600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852c4010a077072696d61727910011801220f636f6e6e656374696f6e5f6e616d652a07637265617465642a07757064617465642a0f636f6e6e656374696f6e5f747970652a12636f6e6e656374696f6e5f64657461696c732a056f776e65722a086f776e65725f6964300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201740a077072696d61727910001a0f636f6e6e656374696f6e5f6e616d651a07637265617465641a07757064617465641a0f636f6e6e656374696f6e5f747970651a12636f6e6e656374696f6e5f64657461696c731a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89be8a89","value":"030aa0040a086a6f625f696e666f1836200128013a00422b0a066a6f625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e666f5f6b657910021a0c0807100018003000501960002000300068007000780080010088010098010042420a077772697474656e10031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a0576616c756510041a0c080810001800300050116000200130006800700078008001008801009801004805528b010a077072696d6172791001180122066a6f625f69642208696e666f5f6b657922077772697474656e2a0576616c75653001300230034000400040014a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201370a077072696d61727910001a066a6f625f69641a08696e666f5f6b65791a077772697474656e1a0576616c756520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89bf8a89","value":"030a9c040a167370616e5f73746174735f756e697175655f6b6579731837200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422e0a096b65795f627974657310021a0c08081000180030005011600020013000680070007800800100880100980100480352700a077072696d61727910011801220269642a096b65795f6279746573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a7e0a19756e697175655f6b6579735f6b65795f62797465735f6964781002180122096b65795f62797465733002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201200a077072696d61727910001a0269641a096b65795f6279746573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c08a89","value":"030aff050a127370616e5f73746174735f6275636b6574731838200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422f0a0973616d706c655f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042320a0c73746172745f6b65795f696410031a0d080e10001800300050861760002000300068007000780080010088010098010042300a0a656e645f6b65795f696410041a0d080e100018003000508617600020003000680070007800800100880100980100422d0a08726571756573747310051a0c080110401800300050146000200030006800700078008001008801009801004806529a010a077072696d61727910011801220269642a0973616d706c655f69642a0c73746172745f6b65795f69642a0a656e645f6b65795f69642a087265717565737473300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7a0a156275636b6574735f73616d706c655f69645f69647810021800220973616d706c655f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2014a0a077072696d61727910001a0269641a0973616d706c655f69641a0c73746172745f6b65795f69641a0a656e645f6b65795f69641a087265717565737473200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c18a89","value":"030ab2040a127370616e5f73746174735f73616d706c65731839200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f757569642829300068007000780080010088010098010042440a0b73616d706c655f74696d6510021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100480352720a077072696d61727910011801220269642a0b73616d706c655f74696d65300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a7e0a1773616d706c65735f73616d706c655f74696d655f69647810021801220b73616d706c655f74696d653002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201220a077072696d61727910001a0269641a0b73616d706c655f74696d65200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c28a89","value":"030aa6030a1c7370616e5f73746174735f74656e616e745f626f756e646172696573183a200128013a00422e0a0974656e616e745f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a626f756e64617269657310021a0c08081000180030005011600020003000680070007800800100880100980100480352780a077072696d61727910011801220974656e616e745f69642a0a626f756e646172696573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201280a077072696d61727910001a0974656e616e745f69641a0a626f756e646172696573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c38a89","value":"030aae060a0d7461736b5f7061796c6f616473183b200128013a0042270a02696410011a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210031a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010042300a0b6d696e5f76657273696f6e10051a0c0807100018003000501960002000300068007000780080010088010098010042300a0b6465736372697074696f6e10061a0c0807100018003000501960002001300068007000780080010088010098010042290a047479706510071a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510081a0c08081000180030005011600020003000680070007800800100880100980100480952b2010a077072696d61727910011801220269642a07637265617465642a056f776e65722a086f776e65725f69642a0b6d696e5f76657273696f6e2a0b6465736372697074696f6e2a04747970652a0576616c7565300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201620a077072696d61727910001a0269641a07637265617465641a056f776e65721a086f776e65725f69641a0b6d696e5f76657273696f6e1a0b6465736372697074696f6e1a04747970651a0576616c7565200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c48a89","value":"030aff050a0c74656e616e745f7461736b73183c200128013a00422e0a0974656e616e745f696410011a0c08011040180030005014600020003000680070007800800100880100980100422b0a0669737375657210021a0c08071000180030005019600020003000680070007800800100880100980100422c0a077461736b5f696410031a0c0801104018003000501460002000300068007000780080010088010098010042420a076372656174656410041a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422f0a0a7061796c6f61645f696410051a0c08071000180030005019600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852b1010a077072696d61727910011801220974656e616e745f6964220669737375657222077461736b5f69642a07637265617465642a0a7061796c6f61645f69642a056f776e65722a086f776e65725f69643001300230034000400040004a10080010001a00200028003000380040005a0070047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2015d0a077072696d61727910001a0974656e616e745f69641a066973737565721a077461736b5f69641a07637265617465641a0a7061796c6f61645f69641a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c58a89","value":"030af1180a1273746174656d656e745f6163746976697479183d200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c0807100018003000501960002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10061a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110071a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310081a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e10091a0d081210001800300050da1d600020003000680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100a1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d300068007000780080010088010098010042340a0f657865637574696f6e5f636f756e74100b1a0c08011040180030005014600020003000680070007800800100880100980100423d0a17657865637574696f6e5f746f74616c5f7365636f6e6473100c1a0d080210401800300050bd0560002000300068007000780080010088010098010042450a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e6473100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e6473100e1a0d080210401800300050bd0560002000300068007000780080010088010098010042370a116370755f73716c5f6176675f6e616e6f73100f1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f6176675f7365636f6e647310101a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f7039395f7365636f6e647310111a0d080210401800300050bd05600020003000680070007800800100880100980100481252cd030a077072696d61727910011801220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d652a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e732a0f657865637574696f6e5f636f756e742a17657865637574696f6e5f746f74616c5f7365636f6e64732a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64732a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64732a116370755f73716c5f6176675f6e616e6f732a1b736572766963655f6c6174656e63795f6176675f7365636f6e64732a1b736572766963655f6c6174656e63795f7039395f7365636f6e647330013002300330043005400040004000400040004a10080010001a00200028003000380040005a007006700770087009700a700b700c700d700e700f701070117a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa0010a1266696e6765727072696e745f69645f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380138043805400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a97010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f7473220f657865637574696f6e5f636f756e743001300b3802380338043805400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aa9010a1b657865637574696f6e5f746f74616c5f7365636f6e64735f69647810041800220d616767726567617465645f74732217657865637574696f6e5f746f74616c5f7365636f6e64733001300c3802380338043805400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64735f69647810051800220d616767726567617465645f7473221b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64733001300e3802380338043805400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a9d010a156370755f73716c5f6176675f6e616e6f735f69647810061800220d616767726567617465645f747322116370755f73716c5f6176675f6e616e6f733001300f3802380338043805400040014a10080010001a00200028003000380040005a00680f7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f736572766963655f6c6174656e63795f6176675f7365636f6e64735f69647810071800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f6176675f7365636f6e6473300130103802380338043805400040014a10080010001a00200028003000380040005a0068107a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f736572766963655f6c6174656e63795f7039395f7365636f6e64735f69647810081800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f7039395f7365636f6e6473300130113802380338043805400040014a10080010001a00200028003000380040005a0068117a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201f5020a077072696d61727910001a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e731a0f657865637574696f6e5f636f756e741a17657865637574696f6e5f746f74616c5f7365636f6e64731a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64731a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64731a116370755f73716c5f6176675f6e616e6f731a1b736572766963655f6c6174656e63795f6176675f7365636f6e64731a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473200120022003200420052006200720082009200a200b200c200d200e200f201020112800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c68a89","value":"030adc150a147472616e73616374696f6e5f6163746976697479183e200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10041a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110051a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310061a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a05717565727910071a0c0807100018003000501960002000300068007000780080010088010098010042340a0f657865637574696f6e5f636f756e7410081a0c08011040180030005014600020003000680070007800800100880100980100423d0a17657865637574696f6e5f746f74616c5f7365636f6e647310091a0d080210401800300050bd0560002000300068007000780080010088010098010042450a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e6473100a1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e6473100b1a0d080210401800300050bd0560002000300068007000780080010088010098010042370a116370755f73716c5f6176675f6e616e6f73100c1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f6176675f7365636f6e6473100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473100e1a0d080210401800300050bd05600020003000680070007800800100880100980100480f5286030a077072696d61727910011801220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d652a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a0571756572792a0f657865637574696f6e5f636f756e742a17657865637574696f6e5f746f74616c5f7365636f6e64732a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64732a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64732a116370755f73716c5f6176675f6e616e6f732a1b736572766963655f6c6174656e63795f6176675f7365636f6e64732a1b736572766963655f6c6174656e63795f7039395f7365636f6e64733001300230034000400040004a10080010001a00200028003000380040005a00700470057006700770087009700a700b700c700d700e7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7e0a1266696e6765727072696e745f69645f69647810021800220e66696e6765727072696e745f696430023801380340004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a93010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f7473220f657865637574696f6e5f636f756e743001300838023803400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aa5010a1b657865637574696f6e5f746f74616c5f7365636f6e64735f69647810041800220d616767726567617465645f74732217657865637574696f6e5f746f74616c5f7365636f6e64733001300938023803400040014a10080010001a00200028003000380040005a0068097a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64735f69647810051800220d616767726567617465645f7473221b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64733001300b38023803400040014a10080010001a00200028003000380040005a00680b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a99010a156370755f73716c5f6176675f6e616e6f735f69647810061800220d616767726567617465645f747322116370755f73716c5f6176675f6e616e6f733001300c38023803400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f736572766963655f6c6174656e63795f6176675f7365636f6e64735f69647810071800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f6176675f7365636f6e64733001300d38023803400040014a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f736572766963655f6c6174656e63795f7039395f7365636f6e64735f69647810081800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f7039395f7365636f6e64733001300e38023803400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201b2020a077072696d61727910001a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a0571756572791a0f657865637574696f6e5f636f756e741a17657865637574696f6e5f746f74616c5f7365636f6e64731a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64731a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64731a116370755f73716c5f6176675f6e616e6f731a1b736572766963655f6c6174656e63795f6176675f7365636f6e64731a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473200120022003200420052006200720082009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c78a89","value":"030adb020a0d74656e616e745f69645f736571183f200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011e0801100118ffffffffffffffff7f20012800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c88a89","value":"030ac50a0a0f6d7663635f737461746973746963731840200128013a0042450a0a637265617465645f617410011a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042300a0b64617461626173655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422d0a087461626c655f696410031a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e6465785f696410041a0c0801104018003000501460002000300068007000780080010088010098010042300a0a7374617469737469637310051a0d081210001800300050da1d60002000300068007000780080010088010098010042ab010a3f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313610061a0c080110201800300050176000200030015a456d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328637265617465645f61742929292c2031363a3a3a494e543829680070007800800101880100980100480752e4020a146d7663635f737461746973746963735f706b657910011801223f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f3136220a637265617465645f6174220b64617461626173655f696422087461626c655f69642208696e6465785f69642a0a7374617469737469637330063001300230033004400040004000400040004a10080010001a00200028003000380040005a0070057a0408002000800100880100900104980101a201720801123f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f31361810220a637265617465645f6174220b64617461626173655f69642208696e6465785f696422087461626c655f6964a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201bd020ae901637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291245636865636b5f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313618002806300038014002b201500a077072696d61727910001a0a637265617465645f61741a0b64617461626173655f69641a087461626c655f69641a08696e6465785f69641a0a73746174697374696373200120022003200420052805b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c98a89","value":"030abb170a1e7472616e73616374696f6e5f657865637574696f6e5f696e7369676874731841200128013a0042340a0e7472616e73616374696f6e5f696410011a0d080e100018003000508617600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410021a0c0808100018003000501160002000300068007000780080010088010098010042320a0d71756572795f73756d6d61727910031a0c0807100018003000501960002001300068007000780080010088010098010042310a0c696d706c696369745f74786e10041a0c08001000180030005010600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410051a0c0807100018003000501960002000300068007000780080010088010098010042300a0a73746172745f74696d6510061a0d080910001800300050a009600020013000680070007800800100880100980100422e0a08656e645f74696d6510071a0d080910001800300050a009600020013000680070007800800100880100980100422e0a09757365725f6e616d6510081a0c08071000180030005019600020013000680070007800800100880100980100422d0a086170705f6e616d6510091a0c0807100018003000501960002001300068007000780080010088010098010042320a0d757365725f7072696f72697479100a1a0c08071000180030005019600020013000680070007800800100880100980100422c0a0772657472696573100b1a0c0801104018003000501460002001300068007000780080010088010098010042360a116c6173745f72657472795f726561736f6e100c1a0c08071000180030005019600020013000680070007800800100880100980100423e0a0870726f626c656d73100d1a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100423c0a06636175736573100e1a1d080f104018003000380150f8075a0c08011040180030005014600060002001300068007000780080010088010098010042480a1273746d745f657865637574696f6e5f696473100f1a1d080f100018003000380750f1075a0c08071000180030005019600060002001300068007000780080010088010098010042320a0d6370755f73716c5f6e616e6f7310101a0c0801104018003000501460002001300068007000780080010088010098010042340a0f6c6173745f6572726f725f636f646510111a0c08071000180030005019600020013000680070007800800100880100980100422b0a0673746174757310121a0c08011040180030005014600020013000680070007800800100880100980100423b0a0f636f6e74656e74696f6e5f74696d6510131a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042350a0f636f6e74656e74696f6e5f696e666f10141a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0764657461696c7310151a0d081210001800300050da1d60002001300068007000780080010088010098010042420a076372656174656410161a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042a0010a2a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313610171a0c080110201800300050176000200030015a4f6d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328656e645f74696d652c2073746172745f74696d652929292c2031363a3a3a494e543829680070007800800101880100980100481852b6030a077072696d61727910011801220e7472616e73616374696f6e5f69642a1a7472616e73616374696f6e5f66696e6765727072696e745f69642a0d71756572795f73756d6d6172792a0c696d706c696369745f74786e2a0a73657373696f6e5f69642a0a73746172745f74696d652a08656e645f74696d652a09757365725f6e616d652a086170705f6e616d652a0d757365725f7072696f726974792a07726574726965732a116c6173745f72657472795f726561736f6e2a0870726f626c656d732a066361757365732a1273746d745f657865637574696f6e5f6964732a0d6370755f73716c5f6e616e6f732a0f6c6173745f6572726f725f636f64652a067374617475732a0f636f6e74656e74696f6e5f74696d652a0f636f6e74656e74696f6e5f696e666f2a0764657461696c732a0763726561746564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b700c700d700e700f70107011701270137014701570167a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a94010a1e7472616e73616374696f6e5f66696e6765727072696e745f69645f69647810021800221a7472616e73616374696f6e5f66696e6765727072696e745f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005af2010a0e74696d655f72616e67655f69647810031800222a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136220a73746172745f74696d652208656e645f74696d6530173006300738014000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a201460801122a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618102208656e645f74696d65220a73746172745f74696d65a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20193020ad401637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291230636865636b5f637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618002817300038014002b201e6020a077072696d61727910001a0e7472616e73616374696f6e5f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a0d71756572795f73756d6d6172791a0c696d706c696369745f74786e1a0a73657373696f6e5f69641a0a73746172745f74696d651a08656e645f74696d651a09757365725f6e616d651a086170705f6e616d651a0d757365725f7072696f726974791a07726574726965731a116c6173745f72657472795f726561736f6e1a0870726f626c656d731a066361757365731a1273746d745f657865637574696f6e5f6964731a0d6370755f73716c5f6e616e6f731a0f6c6173745f6572726f725f636f64651a067374617475731a0f636f6e74656e74696f6e5f74696d651a0f636f6e74656e74696f6e5f696e666f1a0764657461696c731a0763726561746564200120022003200420052006200720082009200a200b200c200d200e200f20102011201220132014201520162800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ca8a89","value":"030a861e0a1c73746174656d656e745f657865637574696f6e5f696e7369676874731842200128013a00422f0a0a73657373696f6e5f696410011a0c0807100018003000501960002000300068007000780080010088010098010042340a0e7472616e73616374696f6e5f696410021a0d080e100018003000508617600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c0808100018003000501160002000300068007000780080010088010098010042310a0c73746174656d656e745f696410041a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f66696e6765727072696e745f696410051a0c08081000180030005011600020003000680070007800800100880100980100422c0a0770726f626c656d10061a0c08011040180030005014600020013000680070007800800100880100980100423c0a0663617573657310071a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a05717565727910081a0c08071000180030005019600020013000680070007800800100880100980100422b0a0673746174757310091a0c0801104018003000501460002001300068007000780080010088010098010042300a0a73746172745f74696d65100a1a0d080910001800300050a009600020013000680070007800800100880100980100422e0a08656e645f74696d65100b1a0d080910001800300050a009600020013000680070007800800100880100980100422e0a0966756c6c5f7363616e100c1a0c08001000180030005010600020013000680070007800800100880100980100422e0a09757365725f6e616d65100d1a0c08071000180030005019600020013000680070007800800100880100980100422d0a086170705f6e616d65100e1a0c0807100018003000501960002001300068007000780080010088010098010042320a0d757365725f7072696f72697479100f1a0c0807100018003000501960002001300068007000780080010088010098010042320a0d64617461626173655f6e616d6510101a0c08071000180030005019600020013000680070007800800100880100980100422e0a09706c616e5f6769737410111a0c08071000180030005019600020013000680070007800800100880100980100422c0a077265747269657310121a0c0801104018003000501460002001300068007000780080010088010098010042360a116c6173745f72657472795f726561736f6e10131a0c0807100018003000501960002001300068007000780080010088010098010042480a12657865637574696f6e5f6e6f64655f69647310141a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100424b0a15696e6465785f7265636f6d6d656e646174696f6e7310151a1d080f100018003000380750f1075a0c08071000180030005019600060002001300068007000780080010088010098010042310a0c696d706c696369745f74786e10161a0c0800100018003000501060002001300068007000780080010088010098010042320a0d6370755f73716c5f6e616e6f7310171a0c08011040180030005014600020013000680070007800800100880100980100422f0a0a6572726f725f636f646510181a0c08071000180030005019600020013000680070007800800100880100980100423b0a0f636f6e74656e74696f6e5f74696d6510191a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042350a0f636f6e74656e74696f6e5f696e666f101a1a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0764657461696c73101b1a0d081210001800300050da1d60002001300068007000780080010088010098010042420a0763726561746564101c1a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042a0010a2a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136101d1a0c080110201800300050176000200030015a4f6d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328656e645f74696d652c2073746172745f74696d652929292c2031363a3a3a494e543829680070007800800101880100980100481e529a040a077072696d61727910011801220c73746174656d656e745f6964220e7472616e73616374696f6e5f69642a0a73657373696f6e5f69642a1a7472616e73616374696f6e5f66696e6765727072696e745f69642a1873746174656d656e745f66696e6765727072696e745f69642a0770726f626c656d2a066361757365732a0571756572792a067374617475732a0a73746172745f74696d652a08656e645f74696d652a0966756c6c5f7363616e2a09757365725f6e616d652a086170705f6e616d652a0d757365725f7072696f726974792a0d64617461626173655f6e616d652a09706c616e5f676973742a07726574726965732a116c6173745f72657472795f726561736f6e2a12657865637574696f6e5f6e6f64655f6964732a15696e6465785f7265636f6d6d656e646174696f6e732a0c696d706c696369745f74786e2a0d6370755f73716c5f6e616e6f732a0a6572726f725f636f64652a0f636f6e74656e74696f6e5f74696d652a0f636f6e74656e74696f6e5f696e666f2a0764657461696c732a076372656174656430043002400040004a10080010001a00200028003000380040005a007001700370057006700770087009700a700b700c700d700e700f7010701170127013701470157016701770187019701a701b701c7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7c0a127472616e73616374696f6e5f69645f69647810021800220e7472616e73616374696f6e5f69643002380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab4010a1e7472616e73616374696f6e5f66696e6765727072696e745f69645f69647810031800221a7472616e73616374696f6e5f66696e6765727072696e745f6964220a73746172745f74696d652208656e645f74696d653003300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab0010a1c73746174656d656e745f66696e6765727072696e745f69645f69647810041800221873746174656d656e745f66696e6765727072696e745f6964220a73746172745f74696d652208656e645f74696d653005300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005af4010a0e74696d655f72616e67655f69647810051800222a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136220a73746172745f74696d652208656e645f74696d65301d300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a201460801122a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618102208656e645f74696d65220a73746172745f74696d65a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20193020ad401637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291230636865636b5f637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f31361800281d300038014002b201c8030a077072696d61727910001a0a73657373696f6e5f69641a0e7472616e73616374696f6e5f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a0c73746174656d656e745f69641a1873746174656d656e745f66696e6765727072696e745f69641a0770726f626c656d1a066361757365731a0571756572791a067374617475731a0a73746172745f74696d651a08656e645f74696d651a0966756c6c5f7363616e1a09757365725f6e616d651a086170705f6e616d651a0d757365725f7072696f726974791a0d64617461626173655f6e616d651a09706c616e5f676973741a07726574726965731a116c6173745f72657472795f726561736f6e1a12657865637574696f6e5f6e6f64655f6964731a15696e6465785f7265636f6d6d656e646174696f6e731a0c696d706c696369745f74786e1a0d6370755f73716c5f6e616e6f731a0a6572726f725f636f64651a0f636f6e74656e74696f6e5f74696d651a0f636f6e74656e74696f6e5f696e666f1a0764657461696c731a0763726561746564200120022003200420052006200720082009200a200b200c200d200e200f2010201120122013201420152016201720182019201a201b201c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89cb8a89","value":"030ad31c0a0e7461626c655f6d657461646174611843200128013a00422a0a0564625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a087461626c655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422c0a0764625f6e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042300a0b736368656d615f6e616d6510041a0c08071000180030005019600020003000680070007800800100880100980100422f0a0a7461626c655f6e616d6510051a0c0807100018003000501960002000300068007000780080010088010098010042320a0d746f74616c5f636f6c756d6e7310061a0c0801104018003000501460002000300068007000780080010088010098010042320a0d746f74616c5f696e646578657310071a0c08011040180030005014600020003000680070007800800100880100980100423f0a0973746f72655f69647310081a1d080f104018003000380150f8075a0c080110401800300050146000600020003000680070007800800100880100980100423b0a167265706c69636174696f6e5f73697a655f627974657310091a0c0801104018003000501460002000300068007000780080010088010098010042310a0c746f74616c5f72616e676573100a1a0c08011040180030005014600020003000680070007800800100880100980100423a0a15746f74616c5f6c6976655f646174615f6279746573100b1a0c0801104018003000501460002000300068007000780080010088010098010042350a10746f74616c5f646174615f6279746573100c1a0c0801104018003000501460002000300068007000780080010088010098010042340a0e706572635f6c6976655f64617461100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042360a116c6173745f7570646174655f6572726f72100e1a0c0807100018003000501960002001300068007000780080010088010098010042470a0c6c6173745f75706461746564100f1a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422f0a0a7461626c655f7479706510101a0c08071000180030005019600020003000680070007800800100880100980100422d0a0764657461696c7310111a0d081210001800300050da1d60002000300068007000780080010088010098010042a4010a2c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313610121a0c080110201800300050176000200030015a516d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f6279746573287461626c655f69642c206c6173745f757064617465642929292c2031363a3a3a494e543829680070007800800101880100980100481352f2020a077072696d61727910011801220564625f696422087461626c655f69642a0764625f6e616d652a0b736368656d615f6e616d652a0a7461626c655f6e616d652a0d746f74616c5f636f6c756d6e732a0d746f74616c5f696e64657865732a0973746f72655f6964732a167265706c69636174696f6e5f73697a655f62797465732a0c746f74616c5f72616e6765732a15746f74616c5f6c6976655f646174615f62797465732a10746f74616c5f646174615f62797465732a0e706572635f6c6976655f646174612a116c6173745f7570646174655f6572726f722a0c6c6173745f757064617465642a0a7461626c655f747970652a0764657461696c7330013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c700d700e700f701070117a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa3010a237265706c69636174696f6e5f73697a655f62797465735f7461626c655f69645f6964781002180022167265706c69636174696f6e5f73697a655f627974657322087461626c655f6964300930023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a8f010a19746f74616c5f72616e6765735f7461626c655f69645f69647810031800220c746f74616c5f72616e67657322087461626c655f6964300a30023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a91010a1a746f74616c5f636f6c756d6e735f7461626c655f69645f69647810041800220d746f74616c5f636f6c756d6e7322087461626c655f6964300630023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a91010a1a746f74616c5f696e64657865735f7461626c655f69645f69647810051800220d746f74616c5f696e646578657322087461626c655f6964300730023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a8f010a15706572635f6c6976655f646174615f69645f69647810061800220e706572635f6c6976655f6461746122087461626c655f6964300d30023801400140004a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005afc010a106c6173745f757064617465645f69647810071800222c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f3136220c6c6173745f7570646174656422087461626c655f69643012300f300238014000400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a2014a0801122c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f31361810220c6c6173745f7570646174656422087461626c655f6964a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a730a0b64625f6e616d655f67696e10081800220764625f6e616d6530033801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a790a0e7461626c655f6e616d655f67696e10091800220a7461626c655f6e616d6530053801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a7b0a0f736368656d615f6e616d655f67696e100a1800220b736368656d615f6e616d6530043801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a770a0d73746f72655f6964735f67696e100b1800220973746f72655f69647330083801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e00100e9010000000000000000600c6a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20197020ad601637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291232636865636b5f637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313618002812300038014002b201a0020a077072696d61727910001a0564625f69641a087461626c655f69641a0764625f6e616d651a0b736368656d615f6e616d651a0a7461626c655f6e616d651a0d746f74616c5f636f6c756d6e731a0d746f74616c5f696e64657865731a0973746f72655f6964731a167265706c69636174696f6e5f73697a655f62797465731a0c746f74616c5f72616e6765731a15746f74616c5f6c6976655f646174615f62797465731a10746f74616c5f646174615f62797465731a0e706572635f6c6976655f646174611a116c6173745f7570646174655f6572726f721a0c6c6173745f757064617465641a0a7461626c655f747970651a0764657461696c73200120022003200420052006200720082009200a200b200c200d200e200f201020112800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89cc8a89","value":"030aad040a0c6a6f625f70726f67726573731844200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422e0a086672616374696f6e10031a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a087265736f6c76656410041a0d080310001800300050a40d6000200130006800700078008001008801009801004805528c010a077072696d6172791001180122066a6f625f696422077772697474656e2a086672616374696f6e2a087265736f6c76656430013002400040014a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2013a0a077072696d61727910001a066a6f625f69641a077772697474656e1a086672616374696f6e1a087265736f6c76656420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89cd8a89","value":"030ab5040a146a6f625f70726f67726573735f686973746f72791845200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422e0a086672616374696f6e10031a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a087265736f6c76656410041a0d080310001800300050a40d6000200130006800700078008001008801009801004805528c010a077072696d6172791001180122066a6f625f696422077772697474656e2a086672616374696f6e2a087265736f6c76656430013002400040014a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2013a0a077072696d61727910001a066a6f625f69641a077772697474656e1a086672616374696f6e1a087265736f6c76656420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ce8a89","value":"030adb030a0a6a6f625f7374617475731846200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422b0a0673746174757310031a0c080710001800300050196000200030006800700078008001008801009801004804527e0a077072696d6172791001180122066a6f625f696422077772697474656e2a0673746174757330013002400040014a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012c0a077072696d61727910001a066a6f625f69641a077772697474656e1a067374617475732001200220032803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89cf8a89","value":"030a9d040a0b6a6f625f6d6573736167651847200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042290a046b696e6410031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076d65737361676510041a0c0807100018003000501960002000300068007000780080010088010098010048055289010a077072696d6172791001180122066a6f625f696422077772697474656e22046b696e642a076d6573736167653001300230034000400140004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201350a077072696d61727910001a066a6f625f69641a077772697474656e1a046b696e641a076d65737361676520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89d08a89","value":"030ab1060a1570726570617265645f7472616e73616374696f6e731848200128013a00422e0a09676c6f62616c5f696410011a0c0807100018003000501960002000300068007000780080010088010098010042340a0e7472616e73616374696f6e5f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042340a0f7472616e73616374696f6e5f6b657910031a0c0808100018003000501160002001300068007000780080010088010098010042430a08707265706172656410041a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210051a0c08071000180030005019600020003000680070007800800100880100980100422d0a08646174616261736510061a0c08071000180030005019600020003000680070007800800100880100980100422e0a0968657572697374696310071a0c08071000180030005019600020013000680070007800800100880100980100480852bd010a077072696d617279100118012209676c6f62616c5f69642a0e7472616e73616374696f6e5f69642a0f7472616e73616374696f6e5f6b65792a0870726570617265642a056f776e65722a0864617461626173652a09686575726973746963300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2016d0a077072696d61727910001a09676c6f62616c5f69641a0e7472616e73616374696f6e5f69641a0f7472616e73616374696f6e5f6b65791a0870726570617265641a056f776e65721a0864617461626173651a0968657572697374696320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} +,{"key":"8b898b8a89","value":"030a97030a0a64657363726970746f721803200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a64657363726970746f7210021a0c08081000180030005011600020013000680070007800800100880100980100480352710a077072696d61727910011801220269642a0a64657363726970746f72300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201130a077072696d61727910001a02696420012800b201240a1066616d5f325f64657363726970746f7210021a0a64657363726970746f7220022802b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b898c8a89","value":"030ad0050a0575736572731804200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042330a0e68617368656450617373776f726410021a0c0808100018003000501160002001300068007000780080010088010098010042320a066973526f6c6510031a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055290010a077072696d617279100118012208757365726e616d652a0e68617368656450617373776f72642a066973526f6c652a07757365725f6964300140004a10080010001a00200028003000380040005a007002700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a740a1175736572735f757365725f69645f696478100218012207757365725f69643004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201240a077072696d61727910001a08757365726e616d651a07757365725f6964200120042804b2012c0a1466616d5f325f68617368656450617373776f726410021a0e68617368656450617373776f726420022802b2011c0a0c66616d5f335f6973526f6c6510031a066973526f6c6520032803b80104c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b898d8a89","value":"030a86030a057a6f6e65731805200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422b0a06636f6e66696710021a0c080810001800300050116000200130006800700078008001008801009801004803526d0a077072696d61727910011801220269642a06636f6e666967300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201130a077072696d61727910001a02696420012800b2011c0a0c66616d5f325f636f6e66696710021a06636f6e66696720022802b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b898e8a89","value":"030ac5040a0873657474696e67731806200128013a0042290a046e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510021a0c0807100018003000501960002000300068007000780080010088010098010042440a0b6c6173745570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422e0a0976616c75655479706510041a0c080710001800300050196000200130006800700078008001008801009801004805528a010a077072696d6172791001180122046e616d652a0576616c75652a0b6c617374557064617465642a0976616c756554797065300140004a10080010001a00200028003000380040005a007002700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201590a2666616d5f305f6e616d655f76616c75655f6c617374557064617465645f76616c75655479706510001a046e616d651a0576616c75651a0b6c617374557064617465641a0976616c75655479706520012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b898f8a89","value":"030ae2020a1164657363726970746f725f69645f7365711807200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011e0801100118ffffffffffffffff7f20012800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89908a89","value":"030a81070a0774656e616e74731808200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042310a0661637469766510021a0c08001000180030005010600020002a0474727565300168007000780080010088010098010042290a04696e666f10031a0c0808100018003000501160002001300068007000780080010088010098010042290a046e616d6510041a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a646174615f737461746510051a0c0801104018003000501460002001300068007000780080010088010098010042310a0c736572766963655f6d6f646510061a0c080110401800300050146000200130006800700078008001008801009801004807529b010a077072696d61727910011801220269642a066163746976652a04696e666f2a046e616d652a0a646174615f73746174652a0c736572766963655f6d6f6465300140004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a700a1074656e616e74735f6e616d655f6964781002180122046e616d653004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a80010a1874656e616e74735f736572766963655f6d6f64655f69647810031800220c736572766963655f6d6f64653006380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060046a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2014b0a077072696d61727910001a0269641a066163746976651a04696e666f1a046e616d651a0a646174615f73746174651a0c736572766963655f6d6f64652001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89918a89","value":"030abd030a0f726567696f6e5f6c6976656e6573731809200128013a0042300a0b637264625f726567696f6e10011a0c0808100018003000501160002000300068007000780080010088010098010042340a0e756e617661696c61626c655f617410021a0d080510001800300050da086000200130006800700078008001008801009801004803528b010a14726567696f6e5f6c6976656e6573735f706b657910011801220b637264625f726567696f6e2a0e756e617661696c61626c655f6174300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012e0a077072696d61727910001a0b637264625f726567696f6e1a0e756e617661696c61626c655f6174200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89938a89","value":"030a83050a056c65617365180b200128013a00422c0a07646573635f696410011a0c08011040180030005014600020003000680070007800800100880100980100422c0a0776657273696f6e10021a0c0801104018003000501460002000300068007000780080010088010098010042340a0f73716c5f696e7374616e63655f696410031a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a73657373696f6e5f696410041a0c0808100018003000501160002000300068007000780080010088010098010042300a0b637264625f726567696f6e10051a0c08081000180030005011600020003000680070007800800100880100980100480652a9010a077072696d61727910031801220b637264625f726567696f6e2207646573635f6964220776657273696f6e220a73657373696f6e5f69642a0f73716c5f696e7374616e63655f6964300530013002300440004000400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201530a077072696d61727910001a07646573635f69641a0776657273696f6e1a0f73716c5f696e7374616e63655f69641a0a73657373696f6e5f69641a0b637264625f726567696f6e200120022003200420052803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89948a89","value":"030a96060a086576656e746c6f67180c200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422e0a096576656e745479706510021a0c08071000180030005019600020003000680070007800800100880100980100422d0a08746172676574494410031a0c0801104018003000501460002000300068007000780080010088010098010042300a0b7265706f7274696e67494410041a0c0801104018003000501460002000300068007000780080010088010098010042290a04696e666f10051a0c0807100018003000501960002001300068007000780080010088010098010042380a08756e69717565494410061a0c08081000180030005011600020002a09757569645f763428293000680070007800800100880100980100480752a8010a077072696d61727910011801220974696d657374616d702208756e6971756549442a096576656e74547970652a0874617267657449442a0b7265706f7274696e6749442a04696e666f30013006400040004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120062800b201220a0f66616d5f325f6576656e745479706510021a096576656e745479706520022802b201200a0e66616d5f335f746172676574494410031a08746172676574494420032803b201260a1166616d5f345f7265706f7274696e67494410041a0b7265706f7274696e67494420042804b201180a0a66616d5f355f696e666f10051a04696e666f20052805b80106c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89958a89","value":"030af5060a0872616e67656c6f67180d200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422c0a0772616e6765494410021a0c08011040180030005014600020003000680070007800800100880100980100422c0a0773746f7265494410031a0c08011040180030005014600020003000680070007800800100880100980100422e0a096576656e745479706510041a0c0807100018003000501960002000300068007000780080010088010098010042310a0c6f7468657252616e6765494410051a0c0801104018003000501460002001300068007000780080010088010098010042290a04696e666f10061a0c08071000180030005019600020013000680070007800800100880100980100423d0a08756e69717565494410071a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100480852b3010a077072696d61727910011801220974696d657374616d702208756e6971756549442a0772616e676549442a0773746f726549442a096576656e74547970652a0c6f7468657252616e676549442a04696e666f30013007400040004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120072800b2011e0a0d66616d5f325f72616e6765494410021a0772616e6765494420022802b2011e0a0d66616d5f335f73746f7265494410031a0773746f7265494420032803b201220a0f66616d5f345f6576656e745479706510041a096576656e745479706520042804b201280a1266616d5f355f6f7468657252616e6765494410051a0c6f7468657252616e6765494420052805b201180a0a66616d5f365f696e666f10061a04696e666f20062806b80107c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89968a89","value":"030aed030a027569180e200128013a0042280a036b657910011a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510021a0c0808100018003000501160002001300068007000780080010088010098010042310a0b6c6173745570646174656410031a0d080510001800300050da086000200030006800700078008001008801009801004804527c0a077072696d6172791001180122036b65792a0576616c75652a0b6c61737455706461746564300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201140a077072696d61727910001a036b657920012800b2011a0a0b66616d5f325f76616c756510021a0576616c756520022802b201260a1166616d5f335f6c6173745570646174656410031a0b6c6173745570646174656420032803b80104c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89978a89","value":"030aaf120a046a6f6273180f200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100422b0a0673746174757310021a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f64726f707065645f7061796c6f616410041a0c0808100018003000501160002001300168007000780080010088010098010042350a1064726f707065645f70726f677265737310051a0c0808100018003000501160002001300168007000780080010088010098010042340a0f637265617465645f62795f7479706510061a0c0807100018003000501960002001300068007000780080010088010098010042320a0d637265617465645f62795f696410071a0c0801104018003000501460002001300068007000780080010088010098010042350a10636c61696d5f73657373696f6e5f696410081a0c0808100018003000501160002001300068007000780080010088010098010042360a11636c61696d5f696e7374616e63655f696410091a0c08011040180030005014600020013000680070007800800100880100980100422d0a086e756d5f72756e73100a1a0c08011040180030005014600020013000680070007800800100880100980100422e0a086c6173745f72756e100b1a0d080510001800300050da08600020013000680070007800800100880100980100422d0a086a6f625f74797065100c1a0c08071000180030005019600020013000680070007800800100880100980100422a0a056f776e6572100d1a0c0807100018003000501960002001300068007000780080010088010098010042300a0b6465736372697074696f6e100e1a0c08071000180030005019600020013000680070007800800100880100980100422e0a096572726f725f6d7367100f1a0c08071000180030005019600020013000680070007800800100880100980100422e0a0866696e697368656410101a0d080910001800300050a009600020013000680070007800800100880100980100481152c1020a077072696d61727910011801220269642a067374617475732a07637265617465642a0f64726f707065645f7061796c6f61642a1064726f707065645f70726f67726573732a0f637265617465645f62795f747970652a0d637265617465645f62795f69642a10636c61696d5f73657373696f6e5f69642a11636c61696d5f696e7374616e63655f69642a086e756d5f72756e732a086c6173745f72756e2a086a6f625f747970652a056f776e65722a0b6465736372697074696f6e2a096572726f725f6d73672a0866696e6973686564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b700c700d700e700f70107a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a86010a176a6f62735f7374617475735f637265617465645f696478100218002206737461747573220763726561746564300230033801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aae010a266a6f62735f637265617465645f62795f747970655f637265617465645f62795f69645f69647810031800220f637265617465645f62795f74797065220d637265617465645f62795f69642a06737461747573300630073801400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ac6020a126a6f62735f72756e5f73746174735f696478100418002210636c61696d5f73657373696f6e5f696422067374617475732207637265617465642a086c6173745f72756e2a086e756d5f72756e732a11636c61696d5f696e7374616e63655f696430083002300338014000400040004a10080010001a00200028003000380040005a00700b700a70097a0408002000800100880100900103980100a20106080012001800a80100b20100ba01810173746174757320494e20282772756e6e696e67273a3a3a535452494e472c2027726576657274696e67273a3a3a535452494e472c202770656e64696e67273a3a3a535452494e472c202770617573652d726571756573746564273a3a3a535452494e472c202763616e63656c2d726571756573746564273a3a3a535452494e4729c00100c80100d00100e00100e90100000000000000005a750a116a6f62735f6a6f625f747970655f6964781005180022086a6f625f74797065300c380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201b4010a1f66616d5f305f69645f7374617475735f637265617465645f7061796c6f616410001a0269641a067374617475731a07637265617465641a0f64726f707065645f7061796c6f61641a0f637265617465645f62795f747970651a0d637265617465645f62795f69641a086a6f625f747970651a056f776e65721a0b6465736372697074696f6e1a096572726f725f6d73671a0866696e6973686564200120022003200420062007200c200d200e200f20102800b201220a0870726f677265737310011a1064726f707065645f70726f677265737320052805b2014c0a05636c61696d10021a10636c61696d5f73657373696f6e5f69641a11636c61696d5f696e7374616e63655f69641a086e756d5f72756e731a086c6173745f72756e20082009200a200b2800b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b899b8a89","value":"030a970c0a0c7765625f73657373696f6e731813200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042310a0c68617368656453656372657410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a08757365726e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042420a0963726561746564417410041a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422f0a0965787069726573417410051a0d080510001800300050da08600020003000680070007800800100880100980100422f0a097265766f6b6564417410061a0d080510001800300050da0860002001300068007000780080010088010098010042430a0a6c61737455736564417410071a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422e0a096175646974496e666f10081a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410091a0c080c100018003000501a600020003000680070007800800100880100980100480a52cc010a077072696d61727910011801220269642a0c6861736865645365637265742a08757365726e616d652a096372656174656441742a096578706972657341742a097265766f6b656441742a0a6c6173745573656441742a096175646974496e666f2a07757365725f6964300140004a10080010001a00200028003000380040005a00700270037004700570067007700870097a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f6578706972657341745f6964781002180022096578706972657341743005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f6372656174656441745f6964781003180022096372656174656441743004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f7265766f6b656441745f6964781004180022097265766f6b656441743006380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a81010a1b7765625f73657373696f6e735f6c6173745573656441745f69647810051800220a6c6173745573656441743007380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201c6010a5166616d5f305f69645f6861736865645365637265745f757365726e616d655f6372656174656441745f6578706972657341745f7265766f6b656441745f6c6173745573656441745f6175646974496e666f10001a0269641a0c6861736865645365637265741a08757365726e616d651a096372656174656441741a096578706972657341741a097265766f6b656441741a0a6c6173745573656441741a096175646974496e666f1a07757365725f69642001200220032004200520062007200820092800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b899c8a89","value":"030ab20a0a107461626c655f737461746973746963731814200128013a00422c0a077461626c65494410011a0c0801104018003000501460002000300068007000780080010088010098010042400a0b737461746973746963494410021a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042290a046e616d6510031a0c08071000180030005019600020013000680070007800800100880100980100423f0a09636f6c756d6e49447310041a1d080f104018003000380150f8075a0c08011040180030005014600060002000300068007000780080010088010098010042420a0963726561746564417410051a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422d0a08726f77436f756e7410061a0c0801104018003000501460002000300068007000780080010088010098010042320a0d64697374696e6374436f756e7410071a0c08011040180030005014600020003000680070007800800100880100980100422e0a096e756c6c436f756e7410081a0c08011040180030005014600020003000680070007800800100880100980100422e0a09686973746f6772616d10091a0c0808100018003000501160002001300068007000780080010088010098010042360a0761766753697a65100a1a0c08011040180030005014600020002a08303a3a3a494e5438300068007000780080010088010098010042350a107061727469616c507265646963617465100b1a0c0807100018003000501960002001300068007000780080010088010098010042340a0f66756c6c5374617469737469634944100c1a0c08011040180030005014600020013000680070007800800100880100980100480d5284020a077072696d6172791001180122077461626c654944220b73746174697374696349442a046e616d652a09636f6c756d6e4944732a096372656174656441742a08726f77436f756e742a0d64697374696e6374436f756e742a096e756c6c436f756e742a09686973746f6772616d2a0761766753697a652a107061727469616c5072656469636174652a0f66756c6c537461746973746963494430013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b20188020a5d66616d5f305f7461626c6549445f73746174697374696349445f6e616d655f636f6c756d6e4944735f6372656174656441745f726f77436f756e745f64697374696e6374436f756e745f6e756c6c436f756e745f686973746f6772616d10001a077461626c6549441a0b73746174697374696349441a046e616d651a09636f6c756d6e4944731a096372656174656441741a08726f77436f756e741a0d64697374696e6374436f756e741a096e756c6c436f756e741a09686973746f6772616d1a0761766753697a651a107061727469616c5072656469636174651a0f66756c6c5374617469737469634944200120022003200420052006200720082009200a200b200c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b899d8a89","value":"030ae6040a096c6f636174696f6e731815200128013a0042300a0b6c6f63616c6974794b657910011a0c0807100018003000501960002000300068007000780080010088010098010042320a0d6c6f63616c69747956616c756510021a0c08071000180030005019600020003000680070007800800100880100980100422e0a086c6174697475646510031a0d0803100f1812300050a40d600020003000680070007800800100880100980100422f0a096c6f6e67697475646510041a0d0803100f1812300050a40d60002000300068007000780080010088010098010048055298010a077072696d61727910011801220b6c6f63616c6974794b6579220d6c6f63616c69747956616c75652a086c617469747564652a096c6f6e67697475646530013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201710a3266616d5f305f6c6f63616c6974794b65795f6c6f63616c69747956616c75655f6c617469747564655f6c6f6e67697475646510001a0b6c6f63616c6974794b65791a0d6c6f63616c69747956616c75651a086c617469747564651a096c6f6e67697475646520012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b899f8a89","value":"030aa50a0a0c726f6c655f6d656d626572731817200128013a0042290a04726f6c6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066d656d62657210021a0c08071000180030005019600020003000680070007800800100880100980100422c0a07697341646d696e10031a0c08001000180030005010600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a600020003000680070007800800100880100980100422e0a096d656d6265725f696410051a0c080c100018003000501a60002000300068007000780080010088010098010048065294010a077072696d617279100118012204726f6c6522066d656d6265722a07697341646d696e2a07726f6c655f69642a096d656d6265725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a750a15726f6c655f6d656d626572735f726f6c655f696478100218002204726f6c653001380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a790a17726f6c655f6d656d626572735f6d656d6265725f6964781003180022066d656d6265723002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7d0a18726f6c655f6d656d626572735f726f6c655f69645f696478100418002207726f6c655f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a81010a1a726f6c655f6d656d626572735f6d656d6265725f69645f6964781005180022096d656d6265725f696430053801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a96010a22726f6c655f6d656d626572735f726f6c655f69645f6d656d6265725f69645f6b6579100618012207726f6c655f696422096d656d6265725f69643004300538013802400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060076a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2011f0a077072696d61727910001a04726f6c651a066d656d626572200120022800b2011e0a0d66616d5f335f697341646d696e10031a07697341646d696e20032803b2011e0a0d66616d5f345f726f6c655f696410041a07726f6c655f696420042804b201220a0f66616d5f355f6d656d6265725f696410051a096d656d6265725f696420052805b80106c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a08a89","value":"030ab1040a08636f6d6d656e74731818200128013a0042290a047479706510011a0c08011040180030005014600020003000680070007800800100880100980100422e0a096f626a6563745f696410021a0c08011040180030005014600020003000680070007800800100880100980100422b0a067375625f696410031a0c08011040180030005014600020003000680070007800800100880100980100422c0a07636f6d6d656e7410041a0c080710001800300050196000200030006800700078008001008801009801004805528b010a077072696d6172791001180122047479706522096f626a6563745f696422067375625f69642a07636f6d6d656e743001300230034000400040004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a330a0d0a0561646d696e10e00318e0030a0c0a067075626c6963102018000a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012c0a077072696d61727910001a04747970651a096f626a6563745f69641a067375625f69642001200220032800b2011e0a0d66616d5f345f636f6d6d656e7410041a07636f6d6d656e7420042804b80105c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a18a89","value":"030aac060a1c7265706c69636174696f6e5f636f6e73747261696e745f73746174731819200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c0801104018003000501460002000300068007000780080010088010098010042290a047479706510031a0c08071000180030005019600020003000680070007800800100880100980100422b0a06636f6e66696710041a0c08071000180030005019600020003000680070007800800100880100980100422e0a097265706f72745f696410051a0c0801104018003000501460002000300068007000780080010088010098010042350a0f76696f6c6174696f6e5f737461727410061a0d080910001800300050a00960002001300068007000780080010088010098010042350a1076696f6c6174696e675f72616e67657310071a0c08011040180030005014600020003000680070007800800100880100980100480852c2010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f69642204747970652206636f6e6669672a097265706f72745f69642a0f76696f6c6174696f6e5f73746172742a1076696f6c6174696e675f72616e676573300130023003300440004000400040004a10080010001a00200028003000380040005a007005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2016c0a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a04747970651a06636f6e6669671a097265706f72745f69641a0f76696f6c6174696f6e5f73746172741a1076696f6c6174696e675f72616e67657320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a28a89","value":"030a95050a1f7265706c69636174696f6e5f637269746963616c5f6c6f63616c6974696573181a200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422d0a086c6f63616c69747910031a0c08071000180030005019600020003000680070007800800100880100980100422e0a097265706f72745f696410041a0c0801104018003000501460002000300068007000780080010088010098010042330a0e61745f7269736b5f72616e67657310051a0c08011040180030005014600020003000680070007800800100880100980100480652a5010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f696422086c6f63616c6974792a097265706f72745f69642a0e61745f7269736b5f72616e6765733001300230034000400040004a10080010001a00200028003000380040005a00700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201510a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a086c6f63616c6974791a097265706f72745f69641a0e61745f7269736b5f72616e676573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a38a89","value":"030a83070a117265706c69636174696f6e5f7374617473181b200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422e0a097265706f72745f696410031a0c0801104018003000501460002000300068007000780080010088010098010042310a0c746f74616c5f72616e67657310041a0c0801104018003000501460002000300068007000780080010088010098010042370a12756e617661696c61626c655f72616e67657310051a0c08011040180030005014600020003000680070007800800100880100980100423c0a17756e6465725f7265706c6963617465645f72616e67657310061a0c08011040180030005014600020003000680070007800800100880100980100423b0a166f7665725f7265706c6963617465645f72616e67657310071a0c08011040180030005014600020003000680070007800800100880100980100480852e0010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f69642a097265706f72745f69642a0c746f74616c5f72616e6765732a12756e617661696c61626c655f72616e6765732a17756e6465725f7265706c6963617465645f72616e6765732a166f7665725f7265706c6963617465645f72616e67657330013002400040004a10080010001a00200028003000380040005a00700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2018e010a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a097265706f72745f69641a0c746f74616c5f72616e6765731a12756e617661696c61626c655f72616e6765731a17756e6465725f7265706c6963617465645f72616e6765731a166f7665725f7265706c6963617465645f72616e67657320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a48a89","value":"030a82030a0c7265706f7274735f6d657461181c200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0967656e65726174656410021a0d080910001800300050a009600020003000680070007800800100880100980100480352700a077072696d61727910011801220269642a0967656e657261746564300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201200a077072696d61727910001a0269641a0967656e657261746564200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a68a89","value":"030aa1040a096e616d657370616365181e200128013a00422d0a08706172656e74494410011a0c0801104018003000501460002000300068007000780080010088010098010042330a0e706172656e74536368656d61494410021a0c0801104018003000501460002000300068007000780080010088010098010042290a046e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042270a02696410041a0c080110401800300050146000200130006800700078008001008801009801004805528d010a077072696d617279100118012208706172656e744944220e706172656e74536368656d61494422046e616d652a0269643001300230034000400040004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201330a077072696d61727910001a08706172656e7449441a0e706172656e74536368656d6149441a046e616d652001200220032800b201140a0866616d5f345f696410041a02696420042804b80105c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a78a89","value":"030aab050a1170726f7465637465645f74735f6d657461181f200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422c0a0776657273696f6e10021a0c0801104018003000501460002000300068007000780080010088010098010042300a0b6e756d5f7265636f72647310031a0c08011040180030005014600020003000680070007800800100880100980100422e0a096e756d5f7370616e7310041a0c0801104018003000501460002000300068007000780080010088010098010042300a0b746f74616c5f627974657310051a0c08011040180030005014600020003000680070007800800100880100980100480652a0010a077072696d61727910011801220973696e676c65746f6e2a0776657273696f6e2a0b6e756d5f7265636f7264732a096e756d5f7370616e732a0b746f74616c5f6279746573300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201260a0973696e676c65746f6e120f636865636b5f73696e676c65746f6e18002801300038004002b201500a077072696d61727910001a0973696e676c65746f6e1a0776657273696f6e1a0b6e756d5f7265636f7264731a096e756d5f7370616e731a0b746f74616c5f6279746573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a88a89","value":"030a8f060a1470726f7465637465645f74735f7265636f7264731820200128013a0042280a02696410011a0d080e10001800300050861760002000300068007000780080010088010098010042280a02747310021a0d080310001800300050a40d600020003000680070007800800100880100980100422e0a096d6574615f7479706510031a0c0807100018003000501960002000300068007000780080010088010098010042290a046d65746110041a0c08081000180030005011600020013000680070007800800100880100980100422e0a096e756d5f7370616e7310051a0c08011040180030005014600020003000680070007800800100880100980100422a0a057370616e7310061a0c0808100018003000501160002000300068007000780080010088010098010042340a08766572696669656410071a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422b0a0674617267657410081a0c08081000180030005011600020013000680070007800800100880100980100480952aa010a077072696d61727910011801220269642a0274732a096d6574615f747970652a046d6574612a096e756d5f7370616e732a057370616e732a0876657269666965642a06746172676574300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2015a0a077072696d61727910001a0269641a0274731a096d6574615f747970651a046d6574611a096e756d5f7370616e731a057370616e731a0876657269666965641a06746172676574200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a98a89","value":"030a87050a0c726f6c655f6f7074696f6e731821200128013a00422d0a08757365726e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066f7074696f6e10021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055289010a077072696d617279100118012208757365726e616d6522066f7074696f6e2a0576616c75652a07757365725f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a760a1175736572735f757365725f69645f696478100218002207757365725f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201370a077072696d61727910001a08757365726e616d651a066f7074696f6e1a0576616c75651a07757365725f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89aa8a89","value":"030add030a1773746174656d656e745f62756e646c655f6368756e6b731822200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042300a0b6465736372697074696f6e10021a0c0807100018003000501960002001300068007000780080010088010098010042290a046461746110031a0c080810001800300050116000200030006800700078008001008801009801004804527a0a077072696d61727910011801220269642a0b6465736372697074696f6e2a0464617461300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012a0a077072696d61727910001a0269641a0b6465736372697074696f6e1a04646174612001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ab8a89","value":"030ae80c0a1e73746174656d656e745f646961676e6f73746963735f72657175657374731823200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042350a09636f6d706c6574656410021a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410031a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f646961676e6f73746963735f696410041a0c0801104018003000501460002001300068007000780080010088010098010042320a0c7265717565737465645f617410051a0d080910001800300050a00960002000300068007000780080010088010098010042410a156d696e5f657865637574696f6e5f6c6174656e637910061a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042300a0a657870697265735f617410071a0d080910001800300050a009600020013000680070007800800100880100980100423a0a1473616d706c696e675f70726f626162696c69747910081a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a09706c616e5f6769737410091a0c0807100018003000501960002001300068007000780080010088010098010042330a0e616e74695f706c616e5f67697374100a1a0c0800100018003000501060002001300068007000780080010088010098010042340a087265646163746564100b1a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100480c529f020a077072696d61727910011801220269642a09636f6d706c657465642a1573746174656d656e745f66696e6765727072696e742a1873746174656d656e745f646961676e6f73746963735f69642a0c7265717565737465645f61742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f676973742a087265646163746564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005afb010a0d636f6d706c657465645f696478100218002209636f6d706c65746564220269642a1573746174656d656e745f66696e6765727072696e742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f676973742a08726564616374656430023001400040004a10080010001a00200028003000380040005a0070037006700770087009700a700b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201620a3a73616d706c696e675f70726f626162696c697479204245545745454e20302e303a3a3a464c4f41543820414e4420312e303a3a3a464c4f415438121a636865636b5f73616d706c696e675f70726f626162696c69747918002808300038004002b201cf010a077072696d61727910001a0269641a09636f6d706c657465641a1573746174656d656e745f66696e6765727072696e741a1873746174656d656e745f646961676e6f73746963735f69641a0c7265717565737465645f61741a156d696e5f657865637574696f6e5f6c6174656e63791a0a657870697265735f61741a1473616d706c696e675f70726f626162696c6974791a09706c616e5f676973741a0e616e74695f706c616e5f676973741a087265646163746564200120022003200420052006200720082009200a200b2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ac8a89","value":"030ac1060a1573746174656d656e745f646961676e6f73746963731824200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410021a0c08071000180030005019600020003000680070007800800100880100980100422e0a0973746174656d656e7410031a0c0807100018003000501960002000300068007000780080010088010098010042320a0c636f6c6c65637465645f617410041a0d080910001800300050a009600020003000680070007800800100880100980100422b0a05747261636510051a0d081210001800300050da1d60002001300068007000780080010088010098010042430a0d62756e646c655f6368756e6b7310061a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a056572726f7210071a0c08071000180030005019600020013000680070007800800100880100980100480852bc010a077072696d61727910011801220269642a1573746174656d656e745f66696e6765727072696e742a0973746174656d656e742a0c636f6c6c65637465645f61742a0574726163652a0d62756e646c655f6368756e6b732a056572726f72300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2016c0a077072696d61727910001a0269641a1573746174656d656e745f66696e6765727072696e741a0973746174656d656e741a0c636f6c6c65637465645f61741a0574726163651a0d62756e646c655f6368756e6b731a056572726f7220012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ad8a89","value":"030ad6090a0e7363686564756c65645f6a6f62731825200128013a0042400a0b7363686564756c655f696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042320a0d7363686564756c655f6e616d6510021a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210041a0c08071000180030005019600020003000680070007800800100880100980100422e0a086e6578745f72756e10051a0d080910001800300050a00960002001300068007000780080010088010098010042330a0e7363686564756c655f737461746510061a0c0808100018003000501160002001300068007000780080010088010098010042320a0d7363686564756c655f6578707210071a0c0807100018003000501960002001300068007000780080010088010098010042350a107363686564756c655f64657461696c7310081a0c0808100018003000501160002001300068007000780080010088010098010042320a0d6578656375746f725f7479706510091a0c0807100018003000501960002000300068007000780080010088010098010042330a0e657865637574696f6e5f61726773100a1a0c08081000180030005011600020003000680070007800800100880100980100480b52f7010a077072696d61727910011801220b7363686564756c655f69642a0d7363686564756c655f6e616d652a07637265617465642a056f776e65722a086e6578745f72756e2a0e7363686564756c655f73746174652a0d7363686564756c655f657870722a107363686564756c655f64657461696c732a0d6578656375746f725f747970652a0e657865637574696f6e5f61726773300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a700a0c6e6578745f72756e5f6964781002180022086e6578745f72756e3005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201380a05736368656410001a0b7363686564756c655f69641a086e6578745f72756e1a0e7363686564756c655f73746174652001200520062800b201780a056f7468657210011a0d7363686564756c655f6e616d651a07637265617465641a056f776e65721a0d7363686564756c655f657870721a107363686564756c655f64657461696c731a0d6578656375746f725f747970651a0e657865637574696f6e5f61726773200220032004200720082009200a2800b80102c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89af8a89","value":"030aef030a0b73716c6c6976656e6573731827200128013a00422f0a0a73657373696f6e5f696410011a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10021a0d080310001800300050a40d60002000300068007000780080010088010098010042300a0b637264625f726567696f6e10031a0c080810001800300050116000200030006800700078008001008801009801004804528a010a077072696d61727910021801220b637264625f726567696f6e220a73657373696f6e5f69642a0a65787069726174696f6e30033001400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201380a077072696d61727910001a0b637264625f726567696f6e1a0a73657373696f6e5f69641a0a65787069726174696f6e2003200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b08a89","value":"030adc040a0a6d6967726174696f6e731828200128013a00422a0a056d616a6f7210011a0c08011040180030005014600020003000680070007800800100880100980100422a0a056d696e6f7210021a0c08011040180030005014600020003000680070007800800100880100980100422a0a05706174636810031a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e7465726e616c10041a0c0801104018003000501460002000300068007000780080010088010098010042320a0c636f6d706c657465645f617410051a0d080910001800300050a0096000200030006800700078008001008801009801004806529a010a077072696d6172791001180122056d616a6f7222056d696e6f72220570617463682208696e7465726e616c2a0c636f6d706c657465645f6174300130023003300440004000400040004a10080010001a00200028003000380040005a0070057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201440a077072696d61727910001a056d616a6f721a056d696e6f721a0570617463681a08696e7465726e616c1a0c636f6d706c657465645f6174200120022003200420052805b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b18a89","value":"030ac6030a0b6a6f696e5f746f6b656e731829200128013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422b0a0673656372657410021a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10031a0d080910001800300050a0096000200030006800700078008001008801009801004804527b0a077072696d61727910011801220269642a067365637265742a0a65787069726174696f6e300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012b0a077072696d61727910001a0269641a067365637265741a0a65787069726174696f6e2001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b28a89","value":"030a92280a1473746174656d656e745f73746174697374696373182a200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410061a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10071a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110081a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310091a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e100a1a0d081210001800300050da1d600020003000680070007800800100880100980100429c020a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38100b1a0c080110201800300050176000200030015a8c016d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f69642c20706c616e5f686173682c207472616e73616374696f6e5f66696e6765727072696e745f696429292c20383a3a3a494e543829680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100c1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d3000680070007800800100880100980100426c0a0d696e64657865735f7573616765100d1a0d081210001800300050da1d6000200130005a3728737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27696e6465786573273a3a3a535452494e4768007000780080010188010098010042710a0f657865637574696f6e5f636f756e74100e1a0c080110401800300050146000200130005a3b2828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27636e74273a3a3a535452494e47293a3a494e5438680070007800800100880100980100428a010a0f736572766963655f6c6174656e6379100f1a0d080210401800300050bd056000200130005a53282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f4154386800700078008001008801009801004297010a0d6370755f73716c5f6e616e6f7310101a0d080210401800300050bd056000200130005a62282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e2763707553514c4e616e6f73273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100429c010a0f636f6e74656e74696f6e5f74696d6510111a0d080210401800300050bd056000200130005a65282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e27636f6e74656e74696f6e54696d65273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f41543868007000780080010088010098010042dc010a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d6510121a0d080210401800300050bd056000200130005a95012828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e3e27636e74273a3a3a535452494e47293a3a464c4f415438202a20282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100428a010a0b7039395f6c6174656e637910131a0d080210401800300050bd056000200130005a57282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e276c6174656e6379496e666f273a3a3a535452494e47292d3e27703939273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100481452c2050a077072696d617279100118012268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e732a0f657865637574696f6e5f636f756e742a0f736572766963655f6c6174656e63792a0d6370755f73716c5f6e616e6f732a0f636f6e74656e74696f6e5f74696d652a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d652a0b7039395f6c6174656e6379300b30013002300330043005300640004000400040004000400040004a10080010001a00200028003000380040005a00700770087009700a700c700e700f70107011701270137a0408002000800100880100900104980101a201c70108011268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f69642209706c616e5f68617368221a7472616e73616374696f6e5f66696e6765727072696e745f6964a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa7010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380b3801380438053806400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a89010a11696e64657865735f75736167655f69647810031800220d696e64657865735f7573616765300d380b38013802380338043805380640004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e00100e90100000000000000005acf010a13657865637574696f6e5f636f756e745f69647810041800220d616767726567617465645f747322086170705f6e616d65220f657865637574696f6e5f636f756e7430013005300e380b38023803380438064000400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ad1010a13736572766963655f6c6174656e63795f69647810051800220d616767726567617465645f747322086170705f6e616d65220f736572766963655f6c6174656e637930013005300f380b38023803380438064000400040014a10080010001a00200028003000380040005a00680f7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a116370755f73716c5f6e616e6f735f69647810061800220d616767726567617465645f747322086170705f6e616d65220d6370755f73716c5f6e616e6f73300130053010380b38023803380438064000400040014a10080010001a00200028003000380040005a0068107a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ad1010a13636f6e74656e74696f6e5f74696d655f69647810071800220d616767726567617465645f747322086170705f6e616d65220f636f6e74656e74696f6e5f74696d65300130053011380b38023803380438064000400040014a10080010001a00200028003000380040005a0068117a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005aef010a22746f74616c5f657374696d617465645f657865637574696f6e5f74696d655f69647810081800220d616767726567617465645f747322086170705f6e616d65221e746f74616c5f657374696d617465645f657865637574696f6e5f74696d65300130053012380b38023803380438064000400040014a10080010001a00200028003000380040005a0068127a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac9010a0f7039395f6c6174656e63795f69647810091800220d616767726567617465645f747322086170705f6e616d65220b7039395f6c6174656e6379300130053013380b38023803380438064000400040014a10080010001a00200028003000380040005a0068137a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e9010000000000000000600a6a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201b9020abc01637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e543829126e636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381800280b300038014002b201a4030a077072696d61727910001a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e731a0f657865637574696f6e5f636f756e741a0f736572766963655f6c6174656e63791a0d6370755f73716c5f6e616e6f731a0f636f6e74656e74696f6e5f74696d651a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d651a0b7039395f6c6174656e6379200b200120022003200420052006200720082009200a200c200e200f20102011201220132800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b38a89","value":"030a98200a167472616e73616374696f6e5f73746174697374696373182b200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410041a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10051a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110061a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310071a0d081210001800300050da1d60002000300068007000780080010088010098010042cf010a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3810081a0c080110201800300050176000200030015a656d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f696429292c20383a3a3a494e54382968007000780080010088010098010042710a0f657865637574696f6e5f636f756e7410091a0c080110401800300050146000200130005a3b2828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27636e74273a3a3a535452494e47293a3a494e5438680070007800800100880100980100428a010a0f736572766963655f6c6174656e6379100a1a0d080210401800300050bd056000200130005a53282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f4154386800700078008001008801009801004297010a0d6370755f73716c5f6e616e6f73100b1a0d080210401800300050bd056000200130005a62282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e2763707553514c4e616e6f73273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100429c010a0f636f6e74656e74696f6e5f74696d65100c1a0d080210401800300050bd056000200130005a65282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e27636f6e74656e74696f6e54696d65273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f41543868007000780080010088010098010042dc010a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d65100d1a0d080210401800300050bd056000200130005a95012828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e3e27636e74273a3a3a535452494e47293a3a464c4f415438202a20282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100428a010a0b7039395f6c6174656e6379100e1a0d080210401800300050bd056000200130005a57282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e276c6174656e6379496e666f273a3a3a535452494e47292d3e27703939273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100480f5280040a077072696d617279100118012243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a0f657865637574696f6e5f636f756e742a0f736572766963655f6c6174656e63792a0d6370755f73716c5f6e616e6f732a0f636f6e74656e74696f6e5f74696d652a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d652a0b7039395f6c6174656e637930083001300230033004400040004000400040004a10080010001a00200028003000380040005a007005700670077009700a700b700c700d700e7a0408002000800100880100900104980101a2017b08011243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f6964a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a85010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f69643002380838013803380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005acb010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f747322086170705f6e616d65220f657865637574696f6e5f636f756e743001300330093808380238044000400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a13736572766963655f6c6174656e63795f69647810041800220d616767726567617465645f747322086170705f6e616d65220f736572766963655f6c6174656e637930013003300a3808380238044000400040014a10080010001a00200028003000380040005a00680a7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac9010a116370755f73716c5f6e616e6f735f69647810051800220d616767726567617465645f747322086170705f6e616d65220d6370755f73716c5f6e616e6f7330013003300b3808380238044000400040014a10080010001a00200028003000380040005a00680b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a13636f6e74656e74696f6e5f74696d655f69647810061800220d616767726567617465645f747322086170705f6e616d65220f636f6e74656e74696f6e5f74696d6530013003300c3808380238044000400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005aeb010a22746f74616c5f657374696d617465645f657865637574696f6e5f74696d655f69647810071800220d616767726567617465645f747322086170705f6e616d65221e746f74616c5f657374696d617465645f657865637574696f6e5f74696d6530013003300d3808380238044000400040014a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac5010a0f7039395f6c6174656e63795f69647810081800220d616767726567617465645f747322086170705f6e616d65220b7039395f6c6174656e637930013003300e3808380238044000400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201ef010a9701637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e5438291249636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3818002808300038014002b201b3020a077072696d61727910001a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a0f657865637574696f6e5f636f756e741a0f736572766963655f6c6174656e63791a0d6370755f73716c5f6e616e6f731a0f636f6e74656e74696f6e5f74696d651a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d651a0b7039395f6c6174656e6379200820012002200320042005200620072009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b48a89","value":"030af6050a1664617461626173655f726f6c655f73657474696e6773182c200128013a0042300a0b64617461626173655f696410011a0c080c100018003000501a600020003000680070007800800100880100980100422e0a09726f6c655f6e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100423e0a0873657474696e677310031a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055292010a077072696d61727910011801220b64617461626173655f69642209726f6c655f6e616d652a0873657474696e67732a07726f6c655f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005aae010a2e64617461626173655f726f6c655f73657474696e67735f64617461626173655f69645f726f6c655f69645f6b657910021801220b64617461626173655f69642207726f6c655f69642a0873657474696e6773300130043802400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201400a077072696d61727910001a0b64617461626173655f69641a09726f6c655f6e616d651a0873657474696e67731a07726f6c655f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b58a89","value":"030ac30b0a0c74656e616e745f7573616765182d200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042300a0b696e7374616e63655f696410021a0c0801104018003000501460002000300068007000780080010088010098010042350a106e6578745f696e7374616e63655f696410031a0c0801104018003000501460002000300068007000780080010088010098010042310a0b6c6173745f75706461746510041a0d080510001800300050da0860002000300068007000780080010088010098010042340a0e72755f62757273745f6c696d697410051a0d080210401800300050bd0560002001300068007000780080010088010098010042340a0e72755f726566696c6c5f7261746510061a0d080210401800300050bd0560002001300068007000780080010088010098010042300a0a72755f63757272656e7410071a0d080210401800300050bd0560002001300068007000780080010088010098010042370a1163757272656e745f73686172655f73756d10081a0d080210401800300050bd0560002001300068007000780080010088010098010042360a11746f74616c5f636f6e73756d7074696f6e10091a0c0808100018003000501160002001300068007000780080010088010098010042330a0e696e7374616e63655f6c65617365100a1a0c0808100018003000501160002001300068007000780080010088010098010042310a0c696e7374616e63655f736571100b1a0c0801104018003000501460002001300068007000780080010088010098010042350a0f696e7374616e63655f736861726573100c1a0d080210401800300050bd0560002001300068007000780080010088010098010042320a0d63757272656e745f7261746573100d1a0c08081000180030005011600020013000680070007800800100880100980100422f0a0a6e6578745f7261746573100e1a0c08081000180030005011600020013000680070007800800100880100980100480f52ce020a077072696d61727910011801220974656e616e745f6964220b696e7374616e63655f69642a106e6578745f696e7374616e63655f69642a0b6c6173745f7570646174652a0e72755f62757273745f6c696d69742a0e72755f726566696c6c5f726174652a0a72755f63757272656e742a1163757272656e745f73686172655f73756d2a11746f74616c5f636f6e73756d7074696f6e2a0e696e7374616e63655f6c656173652a0c696e7374616e63655f7365712a0f696e7374616e63655f7368617265732a0d63757272656e745f72617465732a0a6e6578745f726174657330013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c700d700e7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201fc010a077072696d61727910001a0974656e616e745f69641a0b696e7374616e63655f69641a106e6578745f696e7374616e63655f69641a0b6c6173745f7570646174651a0e72755f62757273745f6c696d69741a0e72755f726566696c6c5f726174651a0a72755f63757272656e741a1163757272656e745f73686172655f73756d1a11746f74616c5f636f6e73756d7074696f6e1a0e696e7374616e63655f6c656173651a0c696e7374616e63655f7365711a0f696e7374616e63655f7368617265731a0d63757272656e745f72617465731a0a6e6578745f7261746573200120022003200420052006200720082009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b68a89","value":"030acb060a0d73716c5f696e7374616e636573182e200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046164647210021a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410031a0c08081000180030005011600020013000680070007800800100880100980100422e0a086c6f63616c69747910041a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0873716c5f6164647210051a0c0807100018003000501960002001300068007000780080010088010098010042300a0b637264625f726567696f6e10061a0c0808100018003000501160002000300068007000780080010088010098010042330a0e62696e6172795f76657273696f6e10071a0c0807100018003000501960002001300068007000780080010088010098010042300a0b69735f647261696e696e6710081a0c08001000180030005010600020013000680070007800800100880100980100480952c3010a077072696d61727910021801220b637264625f726567696f6e220269642a04616464722a0a73657373696f6e5f69642a086c6f63616c6974792a0873716c5f616464722a0e62696e6172795f76657273696f6e2a0b69735f647261696e696e6730063001400040004a10080010001a00200028003000380040005a007002700370047005700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201710a077072696d61727910001a0269641a04616464721a0a73657373696f6e5f69641a086c6f63616c6974791a0873716c5f616464721a0b637264625f726567696f6e1a0e62696e6172795f76657273696f6e1a0b69735f647261696e696e67200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b78a89","value":"030a8a040a137370616e5f636f6e66696775726174696f6e73182f200128013a00422e0a0973746172745f6b657910011a0c08081000180030005011600020003000680070007800800100880100980100422c0a07656e645f6b657910021a0c08081000180030005011600020003000680070007800800100880100980100422b0a06636f6e66696710031a0c080810001800300050116000200030006800700078008001008801009801004804527f0a077072696d61727910011801220973746172745f6b65792a07656e645f6b65792a06636f6e666967300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a2012f0a1373746172745f6b6579203c20656e645f6b6579120c636865636b5f626f756e6473180028012802300038004002b2012f0a077072696d61727910001a0973746172745f6b65791a07656e645f6b65791a06636f6e6669672001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b88a89","value":"030adc020a0b726f6c655f69645f7365711830200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a250a0d0a0561646d696e10a00618a0060a0c0a04726f6f7410a00618a00612046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011a0801106418ffffffff0720642800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ba8a89","value":"030af3050a0f74656e616e745f73657474696e67731832200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c0807100018003000501960002000300068007000780080010088010098010042450a0c6c6173745f7570646174656410041a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422f0a0a76616c75655f7479706510051a0c08071000180030005019600020003000680070007800800100880100980100422b0a06726561736f6e10061a0c08071000180030005019600020013000680070007800800100880100980100480752a5010a077072696d61727910011801220974656e616e745f696422046e616d652a0576616c75652a0c6c6173745f757064617465642a0a76616c75655f747970652a06726561736f6e30013002400040004a10080010001a00200028003000380040005a0070037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b20185010a3966616d5f305f74656e616e745f69645f6e616d655f76616c75655f6c6173745f757064617465645f76616c75655f747970655f726561736f6e10001a0974656e616e745f69641a046e616d651a0576616c75651a0c6c6173745f757064617465641a0a76616c75655f747970651a06726561736f6e2001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89bb8a89","value":"030ac1030a0a7370616e5f636f756e741833200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422f0a0a7370616e5f636f756e7410021a0c08011040180030005014600020003000680070007800800100880100980100480352780a077072696d61727910011801220973696e676c65746f6e2a0a7370616e5f636f756e74300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201210a0973696e676c65746f6e120a73696e676c655f726f7718002801300038004002b201280a077072696d61727910001a0973696e676c65746f6e1a0a7370616e5f636f756e74200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89bc8a89","value":"030ae2070a0a70726976696c656765731834200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042290a047061746810021a0c0807100018003000501960002000300068007000780080010088010098010042400a0a70726976696c6567657310031a1d080f100018003000380750f1075a0c08071000180030005019600060002000300068007000780080010088010098010042430a0d6772616e745f6f7074696f6e7310041a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07757365725f696410051a0c080c100018003000501a6000200030006800700078008001008801009801004806529d010a077072696d617279100118012208757365726e616d652204706174682a0a70726976696c656765732a0d6772616e745f6f7074696f6e732a07757365725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00103e00100e90100000000000000005aa7010a1b70726976696c656765735f706174685f757365725f69645f6b6579100218012204706174682207757365725f69642a0a70726976696c656765732a0d6772616e745f6f7074696f6e73300230053801400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa7010a1c70726976696c656765735f706174685f757365726e616d655f6b6579100318012204706174682208757365726e616d652a0a70726976696c656765732a0d6772616e745f6f7074696f6e7330023001400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2014b0a077072696d61727910001a08757365726e616d651a04706174681a0a70726976696c656765731a0d6772616e745f6f7074696f6e731a07757365725f6964200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880304a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89bd8a89","value":"030add060a1465787465726e616c5f636f6e6e656374696f6e731835200128013a0042340a0f636f6e6e656374696f6e5f6e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042400a077570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f636f6e6e656374696f6e5f7479706510041a0c0807100018003000501960002000300068007000780080010088010098010042370a12636f6e6e656374696f6e5f64657461696c7310051a0c08081000180030005011600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852c4010a077072696d61727910011801220f636f6e6e656374696f6e5f6e616d652a07637265617465642a07757064617465642a0f636f6e6e656374696f6e5f747970652a12636f6e6e656374696f6e5f64657461696c732a056f776e65722a086f776e65725f6964300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201740a077072696d61727910001a0f636f6e6e656374696f6e5f6e616d651a07637265617465641a07757064617465641a0f636f6e6e656374696f6e5f747970651a12636f6e6e656374696f6e5f64657461696c731a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89be8a89","value":"030aa3040a086a6f625f696e666f1836200128013a00422b0a066a6f625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e666f5f6b657910021a0c0807100018003000501960002000300068007000780080010088010098010042420a077772697474656e10031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a0576616c756510041a0c080810001800300050116000200130006800700078008001008801009801004805528b010a077072696d6172791001180122066a6f625f69642208696e666f5f6b657922077772697474656e2a0576616c75653001300230034000400040014a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201370a077072696d61727910001a066a6f625f69641a08696e666f5f6b65791a077772697474656e1a0576616c756520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89bf8a89","value":"030a9f040a167370616e5f73746174735f756e697175655f6b6579731837200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422e0a096b65795f627974657310021a0c08081000180030005011600020013000680070007800800100880100980100480352700a077072696d61727910011801220269642a096b65795f6279746573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a7e0a19756e697175655f6b6579735f6b65795f62797465735f6964781002180122096b65795f62797465733002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201200a077072696d61727910001a0269641a096b65795f6279746573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c08a89","value":"030a82060a127370616e5f73746174735f6275636b6574731838200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422f0a0973616d706c655f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042320a0c73746172745f6b65795f696410031a0d080e10001800300050861760002000300068007000780080010088010098010042300a0a656e645f6b65795f696410041a0d080e100018003000508617600020003000680070007800800100880100980100422d0a08726571756573747310051a0c080110401800300050146000200030006800700078008001008801009801004806529a010a077072696d61727910011801220269642a0973616d706c655f69642a0c73746172745f6b65795f69642a0a656e645f6b65795f69642a087265717565737473300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7a0a156275636b6574735f73616d706c655f69645f69647810021800220973616d706c655f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2014a0a077072696d61727910001a0269641a0973616d706c655f69641a0c73746172745f6b65795f69641a0a656e645f6b65795f69641a087265717565737473200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c18a89","value":"030ab5040a127370616e5f73746174735f73616d706c65731839200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f757569642829300068007000780080010088010098010042440a0b73616d706c655f74696d6510021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100480352720a077072696d61727910011801220269642a0b73616d706c655f74696d65300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a7e0a1773616d706c65735f73616d706c655f74696d655f69647810021801220b73616d706c655f74696d653002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201220a077072696d61727910001a0269641a0b73616d706c655f74696d65200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c28a89","value":"030aa9030a1c7370616e5f73746174735f74656e616e745f626f756e646172696573183a200128013a00422e0a0974656e616e745f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a626f756e64617269657310021a0c08081000180030005011600020003000680070007800800100880100980100480352780a077072696d61727910011801220974656e616e745f69642a0a626f756e646172696573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201280a077072696d61727910001a0974656e616e745f69641a0a626f756e646172696573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c38a89","value":"030ab1060a0d7461736b5f7061796c6f616473183b200128013a0042270a02696410011a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210031a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010042300a0b6d696e5f76657273696f6e10051a0c0807100018003000501960002000300068007000780080010088010098010042300a0b6465736372697074696f6e10061a0c0807100018003000501960002001300068007000780080010088010098010042290a047479706510071a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510081a0c08081000180030005011600020003000680070007800800100880100980100480952b2010a077072696d61727910011801220269642a07637265617465642a056f776e65722a086f776e65725f69642a0b6d696e5f76657273696f6e2a0b6465736372697074696f6e2a04747970652a0576616c7565300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201620a077072696d61727910001a0269641a07637265617465641a056f776e65721a086f776e65725f69641a0b6d696e5f76657273696f6e1a0b6465736372697074696f6e1a04747970651a0576616c7565200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c48a89","value":"030a82060a0c74656e616e745f7461736b73183c200128013a00422e0a0974656e616e745f696410011a0c08011040180030005014600020003000680070007800800100880100980100422b0a0669737375657210021a0c08071000180030005019600020003000680070007800800100880100980100422c0a077461736b5f696410031a0c0801104018003000501460002000300068007000780080010088010098010042420a076372656174656410041a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422f0a0a7061796c6f61645f696410051a0c08071000180030005019600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852b1010a077072696d61727910011801220974656e616e745f6964220669737375657222077461736b5f69642a07637265617465642a0a7061796c6f61645f69642a056f776e65722a086f776e65725f69643001300230034000400040004a10080010001a00200028003000380040005a0070047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2015d0a077072696d61727910001a0974656e616e745f69641a066973737565721a077461736b5f69641a07637265617465641a0a7061796c6f61645f69641a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c58a89","value":"030af4180a1273746174656d656e745f6163746976697479183d200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c0807100018003000501960002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10061a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110071a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310081a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e10091a0d081210001800300050da1d600020003000680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100a1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d300068007000780080010088010098010042340a0f657865637574696f6e5f636f756e74100b1a0c08011040180030005014600020003000680070007800800100880100980100423d0a17657865637574696f6e5f746f74616c5f7365636f6e6473100c1a0d080210401800300050bd0560002000300068007000780080010088010098010042450a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e6473100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e6473100e1a0d080210401800300050bd0560002000300068007000780080010088010098010042370a116370755f73716c5f6176675f6e616e6f73100f1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f6176675f7365636f6e647310101a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f7039395f7365636f6e647310111a0d080210401800300050bd05600020003000680070007800800100880100980100481252cd030a077072696d61727910011801220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d652a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e732a0f657865637574696f6e5f636f756e742a17657865637574696f6e5f746f74616c5f7365636f6e64732a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64732a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64732a116370755f73716c5f6176675f6e616e6f732a1b736572766963655f6c6174656e63795f6176675f7365636f6e64732a1b736572766963655f6c6174656e63795f7039395f7365636f6e647330013002300330043005400040004000400040004a10080010001a00200028003000380040005a007006700770087009700a700b700c700d700e700f701070117a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa0010a1266696e6765727072696e745f69645f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380138043805400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a97010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f7473220f657865637574696f6e5f636f756e743001300b3802380338043805400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aa9010a1b657865637574696f6e5f746f74616c5f7365636f6e64735f69647810041800220d616767726567617465645f74732217657865637574696f6e5f746f74616c5f7365636f6e64733001300c3802380338043805400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64735f69647810051800220d616767726567617465645f7473221b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64733001300e3802380338043805400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a9d010a156370755f73716c5f6176675f6e616e6f735f69647810061800220d616767726567617465645f747322116370755f73716c5f6176675f6e616e6f733001300f3802380338043805400040014a10080010001a00200028003000380040005a00680f7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f736572766963655f6c6174656e63795f6176675f7365636f6e64735f69647810071800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f6176675f7365636f6e6473300130103802380338043805400040014a10080010001a00200028003000380040005a0068107a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f736572766963655f6c6174656e63795f7039395f7365636f6e64735f69647810081800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f7039395f7365636f6e6473300130113802380338043805400040014a10080010001a00200028003000380040005a0068117a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201f5020a077072696d61727910001a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e731a0f657865637574696f6e5f636f756e741a17657865637574696f6e5f746f74616c5f7365636f6e64731a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64731a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64731a116370755f73716c5f6176675f6e616e6f731a1b736572766963655f6c6174656e63795f6176675f7365636f6e64731a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473200120022003200420052006200720082009200a200b200c200d200e200f201020112800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c68a89","value":"030adf150a147472616e73616374696f6e5f6163746976697479183e200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10041a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110051a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310061a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a05717565727910071a0c0807100018003000501960002000300068007000780080010088010098010042340a0f657865637574696f6e5f636f756e7410081a0c08011040180030005014600020003000680070007800800100880100980100423d0a17657865637574696f6e5f746f74616c5f7365636f6e647310091a0d080210401800300050bd0560002000300068007000780080010088010098010042450a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e6473100a1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e6473100b1a0d080210401800300050bd0560002000300068007000780080010088010098010042370a116370755f73716c5f6176675f6e616e6f73100c1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f6176675f7365636f6e6473100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473100e1a0d080210401800300050bd05600020003000680070007800800100880100980100480f5286030a077072696d61727910011801220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d652a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a0571756572792a0f657865637574696f6e5f636f756e742a17657865637574696f6e5f746f74616c5f7365636f6e64732a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64732a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64732a116370755f73716c5f6176675f6e616e6f732a1b736572766963655f6c6174656e63795f6176675f7365636f6e64732a1b736572766963655f6c6174656e63795f7039395f7365636f6e64733001300230034000400040004a10080010001a00200028003000380040005a00700470057006700770087009700a700b700c700d700e7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7e0a1266696e6765727072696e745f69645f69647810021800220e66696e6765727072696e745f696430023801380340004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a93010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f7473220f657865637574696f6e5f636f756e743001300838023803400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aa5010a1b657865637574696f6e5f746f74616c5f7365636f6e64735f69647810041800220d616767726567617465645f74732217657865637574696f6e5f746f74616c5f7365636f6e64733001300938023803400040014a10080010001a00200028003000380040005a0068097a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64735f69647810051800220d616767726567617465645f7473221b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64733001300b38023803400040014a10080010001a00200028003000380040005a00680b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a99010a156370755f73716c5f6176675f6e616e6f735f69647810061800220d616767726567617465645f747322116370755f73716c5f6176675f6e616e6f733001300c38023803400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f736572766963655f6c6174656e63795f6176675f7365636f6e64735f69647810071800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f6176675f7365636f6e64733001300d38023803400040014a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f736572766963655f6c6174656e63795f7039395f7365636f6e64735f69647810081800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f7039395f7365636f6e64733001300e38023803400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201b2020a077072696d61727910001a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a0571756572791a0f657865637574696f6e5f636f756e741a17657865637574696f6e5f746f74616c5f7365636f6e64731a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64731a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64731a116370755f73716c5f6176675f6e616e6f731a1b736572766963655f6c6174656e63795f6176675f7365636f6e64731a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473200120022003200420052006200720082009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c78a89","value":"030ade020a0d74656e616e745f69645f736571183f200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011e0801100118ffffffffffffffff7f20012800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c88a89","value":"030ac80a0a0f6d7663635f737461746973746963731840200128013a0042450a0a637265617465645f617410011a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042300a0b64617461626173655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422d0a087461626c655f696410031a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e6465785f696410041a0c0801104018003000501460002000300068007000780080010088010098010042300a0a7374617469737469637310051a0d081210001800300050da1d60002000300068007000780080010088010098010042ab010a3f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313610061a0c080110201800300050176000200030015a456d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328637265617465645f61742929292c2031363a3a3a494e543829680070007800800101880100980100480752e4020a146d7663635f737461746973746963735f706b657910011801223f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f3136220a637265617465645f6174220b64617461626173655f696422087461626c655f69642208696e6465785f69642a0a7374617469737469637330063001300230033004400040004000400040004a10080010001a00200028003000380040005a0070057a0408002000800100880100900104980101a201720801123f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f31361810220a637265617465645f6174220b64617461626173655f69642208696e6465785f696422087461626c655f6964a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201bd020ae901637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291245636865636b5f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313618002806300038014002b201500a077072696d61727910001a0a637265617465645f61741a0b64617461626173655f69641a087461626c655f69641a08696e6465785f69641a0a73746174697374696373200120022003200420052805b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c98a89","value":"030abe170a1e7472616e73616374696f6e5f657865637574696f6e5f696e7369676874731841200128013a0042340a0e7472616e73616374696f6e5f696410011a0d080e100018003000508617600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410021a0c0808100018003000501160002000300068007000780080010088010098010042320a0d71756572795f73756d6d61727910031a0c0807100018003000501960002001300068007000780080010088010098010042310a0c696d706c696369745f74786e10041a0c08001000180030005010600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410051a0c0807100018003000501960002000300068007000780080010088010098010042300a0a73746172745f74696d6510061a0d080910001800300050a009600020013000680070007800800100880100980100422e0a08656e645f74696d6510071a0d080910001800300050a009600020013000680070007800800100880100980100422e0a09757365725f6e616d6510081a0c08071000180030005019600020013000680070007800800100880100980100422d0a086170705f6e616d6510091a0c0807100018003000501960002001300068007000780080010088010098010042320a0d757365725f7072696f72697479100a1a0c08071000180030005019600020013000680070007800800100880100980100422c0a0772657472696573100b1a0c0801104018003000501460002001300068007000780080010088010098010042360a116c6173745f72657472795f726561736f6e100c1a0c08071000180030005019600020013000680070007800800100880100980100423e0a0870726f626c656d73100d1a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100423c0a06636175736573100e1a1d080f104018003000380150f8075a0c08011040180030005014600060002001300068007000780080010088010098010042480a1273746d745f657865637574696f6e5f696473100f1a1d080f100018003000380750f1075a0c08071000180030005019600060002001300068007000780080010088010098010042320a0d6370755f73716c5f6e616e6f7310101a0c0801104018003000501460002001300068007000780080010088010098010042340a0f6c6173745f6572726f725f636f646510111a0c08071000180030005019600020013000680070007800800100880100980100422b0a0673746174757310121a0c08011040180030005014600020013000680070007800800100880100980100423b0a0f636f6e74656e74696f6e5f74696d6510131a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042350a0f636f6e74656e74696f6e5f696e666f10141a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0764657461696c7310151a0d081210001800300050da1d60002001300068007000780080010088010098010042420a076372656174656410161a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042a0010a2a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313610171a0c080110201800300050176000200030015a4f6d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328656e645f74696d652c2073746172745f74696d652929292c2031363a3a3a494e543829680070007800800101880100980100481852b6030a077072696d61727910011801220e7472616e73616374696f6e5f69642a1a7472616e73616374696f6e5f66696e6765727072696e745f69642a0d71756572795f73756d6d6172792a0c696d706c696369745f74786e2a0a73657373696f6e5f69642a0a73746172745f74696d652a08656e645f74696d652a09757365725f6e616d652a086170705f6e616d652a0d757365725f7072696f726974792a07726574726965732a116c6173745f72657472795f726561736f6e2a0870726f626c656d732a066361757365732a1273746d745f657865637574696f6e5f6964732a0d6370755f73716c5f6e616e6f732a0f6c6173745f6572726f725f636f64652a067374617475732a0f636f6e74656e74696f6e5f74696d652a0f636f6e74656e74696f6e5f696e666f2a0764657461696c732a0763726561746564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b700c700d700e700f70107011701270137014701570167a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a94010a1e7472616e73616374696f6e5f66696e6765727072696e745f69645f69647810021800221a7472616e73616374696f6e5f66696e6765727072696e745f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005af2010a0e74696d655f72616e67655f69647810031800222a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136220a73746172745f74696d652208656e645f74696d6530173006300738014000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a201460801122a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618102208656e645f74696d65220a73746172745f74696d65a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20193020ad401637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291230636865636b5f637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618002817300038014002b201e6020a077072696d61727910001a0e7472616e73616374696f6e5f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a0d71756572795f73756d6d6172791a0c696d706c696369745f74786e1a0a73657373696f6e5f69641a0a73746172745f74696d651a08656e645f74696d651a09757365725f6e616d651a086170705f6e616d651a0d757365725f7072696f726974791a07726574726965731a116c6173745f72657472795f726561736f6e1a0870726f626c656d731a066361757365731a1273746d745f657865637574696f6e5f6964731a0d6370755f73716c5f6e616e6f731a0f6c6173745f6572726f725f636f64651a067374617475731a0f636f6e74656e74696f6e5f74696d651a0f636f6e74656e74696f6e5f696e666f1a0764657461696c731a0763726561746564200120022003200420052006200720082009200a200b200c200d200e200f20102011201220132014201520162800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ca8a89","value":"030a891e0a1c73746174656d656e745f657865637574696f6e5f696e7369676874731842200128013a00422f0a0a73657373696f6e5f696410011a0c0807100018003000501960002000300068007000780080010088010098010042340a0e7472616e73616374696f6e5f696410021a0d080e100018003000508617600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c0808100018003000501160002000300068007000780080010088010098010042310a0c73746174656d656e745f696410041a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f66696e6765727072696e745f696410051a0c08081000180030005011600020003000680070007800800100880100980100422c0a0770726f626c656d10061a0c08011040180030005014600020013000680070007800800100880100980100423c0a0663617573657310071a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a05717565727910081a0c08071000180030005019600020013000680070007800800100880100980100422b0a0673746174757310091a0c0801104018003000501460002001300068007000780080010088010098010042300a0a73746172745f74696d65100a1a0d080910001800300050a009600020013000680070007800800100880100980100422e0a08656e645f74696d65100b1a0d080910001800300050a009600020013000680070007800800100880100980100422e0a0966756c6c5f7363616e100c1a0c08001000180030005010600020013000680070007800800100880100980100422e0a09757365725f6e616d65100d1a0c08071000180030005019600020013000680070007800800100880100980100422d0a086170705f6e616d65100e1a0c0807100018003000501960002001300068007000780080010088010098010042320a0d757365725f7072696f72697479100f1a0c0807100018003000501960002001300068007000780080010088010098010042320a0d64617461626173655f6e616d6510101a0c08071000180030005019600020013000680070007800800100880100980100422e0a09706c616e5f6769737410111a0c08071000180030005019600020013000680070007800800100880100980100422c0a077265747269657310121a0c0801104018003000501460002001300068007000780080010088010098010042360a116c6173745f72657472795f726561736f6e10131a0c0807100018003000501960002001300068007000780080010088010098010042480a12657865637574696f6e5f6e6f64655f69647310141a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100424b0a15696e6465785f7265636f6d6d656e646174696f6e7310151a1d080f100018003000380750f1075a0c08071000180030005019600060002001300068007000780080010088010098010042310a0c696d706c696369745f74786e10161a0c0800100018003000501060002001300068007000780080010088010098010042320a0d6370755f73716c5f6e616e6f7310171a0c08011040180030005014600020013000680070007800800100880100980100422f0a0a6572726f725f636f646510181a0c08071000180030005019600020013000680070007800800100880100980100423b0a0f636f6e74656e74696f6e5f74696d6510191a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042350a0f636f6e74656e74696f6e5f696e666f101a1a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0764657461696c73101b1a0d081210001800300050da1d60002001300068007000780080010088010098010042420a0763726561746564101c1a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042a0010a2a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136101d1a0c080110201800300050176000200030015a4f6d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328656e645f74696d652c2073746172745f74696d652929292c2031363a3a3a494e543829680070007800800101880100980100481e529a040a077072696d61727910011801220c73746174656d656e745f6964220e7472616e73616374696f6e5f69642a0a73657373696f6e5f69642a1a7472616e73616374696f6e5f66696e6765727072696e745f69642a1873746174656d656e745f66696e6765727072696e745f69642a0770726f626c656d2a066361757365732a0571756572792a067374617475732a0a73746172745f74696d652a08656e645f74696d652a0966756c6c5f7363616e2a09757365725f6e616d652a086170705f6e616d652a0d757365725f7072696f726974792a0d64617461626173655f6e616d652a09706c616e5f676973742a07726574726965732a116c6173745f72657472795f726561736f6e2a12657865637574696f6e5f6e6f64655f6964732a15696e6465785f7265636f6d6d656e646174696f6e732a0c696d706c696369745f74786e2a0d6370755f73716c5f6e616e6f732a0a6572726f725f636f64652a0f636f6e74656e74696f6e5f74696d652a0f636f6e74656e74696f6e5f696e666f2a0764657461696c732a076372656174656430043002400040004a10080010001a00200028003000380040005a007001700370057006700770087009700a700b700c700d700e700f7010701170127013701470157016701770187019701a701b701c7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7c0a127472616e73616374696f6e5f69645f69647810021800220e7472616e73616374696f6e5f69643002380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab4010a1e7472616e73616374696f6e5f66696e6765727072696e745f69645f69647810031800221a7472616e73616374696f6e5f66696e6765727072696e745f6964220a73746172745f74696d652208656e645f74696d653003300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab0010a1c73746174656d656e745f66696e6765727072696e745f69645f69647810041800221873746174656d656e745f66696e6765727072696e745f6964220a73746172745f74696d652208656e645f74696d653005300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005af4010a0e74696d655f72616e67655f69647810051800222a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136220a73746172745f74696d652208656e645f74696d65301d300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a201460801122a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618102208656e645f74696d65220a73746172745f74696d65a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20193020ad401637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291230636865636b5f637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f31361800281d300038014002b201c8030a077072696d61727910001a0a73657373696f6e5f69641a0e7472616e73616374696f6e5f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a0c73746174656d656e745f69641a1873746174656d656e745f66696e6765727072696e745f69641a0770726f626c656d1a066361757365731a0571756572791a067374617475731a0a73746172745f74696d651a08656e645f74696d651a0966756c6c5f7363616e1a09757365725f6e616d651a086170705f6e616d651a0d757365725f7072696f726974791a0d64617461626173655f6e616d651a09706c616e5f676973741a07726574726965731a116c6173745f72657472795f726561736f6e1a12657865637574696f6e5f6e6f64655f6964731a15696e6465785f7265636f6d6d656e646174696f6e731a0c696d706c696369745f74786e1a0d6370755f73716c5f6e616e6f731a0a6572726f725f636f64651a0f636f6e74656e74696f6e5f74696d651a0f636f6e74656e74696f6e5f696e666f1a0764657461696c731a0763726561746564200120022003200420052006200720082009200a200b200c200d200e200f2010201120122013201420152016201720182019201a201b201c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89cb8a89","value":"030ad61c0a0e7461626c655f6d657461646174611843200128013a00422a0a0564625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a087461626c655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422c0a0764625f6e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042300a0b736368656d615f6e616d6510041a0c08071000180030005019600020003000680070007800800100880100980100422f0a0a7461626c655f6e616d6510051a0c0807100018003000501960002000300068007000780080010088010098010042320a0d746f74616c5f636f6c756d6e7310061a0c0801104018003000501460002000300068007000780080010088010098010042320a0d746f74616c5f696e646578657310071a0c08011040180030005014600020003000680070007800800100880100980100423f0a0973746f72655f69647310081a1d080f104018003000380150f8075a0c080110401800300050146000600020003000680070007800800100880100980100423b0a167265706c69636174696f6e5f73697a655f627974657310091a0c0801104018003000501460002000300068007000780080010088010098010042310a0c746f74616c5f72616e676573100a1a0c08011040180030005014600020003000680070007800800100880100980100423a0a15746f74616c5f6c6976655f646174615f6279746573100b1a0c0801104018003000501460002000300068007000780080010088010098010042350a10746f74616c5f646174615f6279746573100c1a0c0801104018003000501460002000300068007000780080010088010098010042340a0e706572635f6c6976655f64617461100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042360a116c6173745f7570646174655f6572726f72100e1a0c0807100018003000501960002001300068007000780080010088010098010042470a0c6c6173745f75706461746564100f1a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422f0a0a7461626c655f7479706510101a0c08071000180030005019600020003000680070007800800100880100980100422d0a0764657461696c7310111a0d081210001800300050da1d60002000300068007000780080010088010098010042a4010a2c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313610121a0c080110201800300050176000200030015a516d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f6279746573287461626c655f69642c206c6173745f757064617465642929292c2031363a3a3a494e543829680070007800800101880100980100481352f2020a077072696d61727910011801220564625f696422087461626c655f69642a0764625f6e616d652a0b736368656d615f6e616d652a0a7461626c655f6e616d652a0d746f74616c5f636f6c756d6e732a0d746f74616c5f696e64657865732a0973746f72655f6964732a167265706c69636174696f6e5f73697a655f62797465732a0c746f74616c5f72616e6765732a15746f74616c5f6c6976655f646174615f62797465732a10746f74616c5f646174615f62797465732a0e706572635f6c6976655f646174612a116c6173745f7570646174655f6572726f722a0c6c6173745f757064617465642a0a7461626c655f747970652a0764657461696c7330013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c700d700e700f701070117a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa3010a237265706c69636174696f6e5f73697a655f62797465735f7461626c655f69645f6964781002180022167265706c69636174696f6e5f73697a655f627974657322087461626c655f6964300930023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a8f010a19746f74616c5f72616e6765735f7461626c655f69645f69647810031800220c746f74616c5f72616e67657322087461626c655f6964300a30023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a91010a1a746f74616c5f636f6c756d6e735f7461626c655f69645f69647810041800220d746f74616c5f636f6c756d6e7322087461626c655f6964300630023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a91010a1a746f74616c5f696e64657865735f7461626c655f69645f69647810051800220d746f74616c5f696e646578657322087461626c655f6964300730023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a8f010a15706572635f6c6976655f646174615f69645f69647810061800220e706572635f6c6976655f6461746122087461626c655f6964300d30023801400140004a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005afc010a106c6173745f757064617465645f69647810071800222c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f3136220c6c6173745f7570646174656422087461626c655f69643012300f300238014000400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a2014a0801122c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f31361810220c6c6173745f7570646174656422087461626c655f6964a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a730a0b64625f6e616d655f67696e10081800220764625f6e616d6530033801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a790a0e7461626c655f6e616d655f67696e10091800220a7461626c655f6e616d6530053801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a7b0a0f736368656d615f6e616d655f67696e100a1800220b736368656d615f6e616d6530043801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a770a0d73746f72655f6964735f67696e100b1800220973746f72655f69647330083801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e00100e9010000000000000000600c6a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20197020ad601637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291232636865636b5f637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313618002812300038014002b201a0020a077072696d61727910001a0564625f69641a087461626c655f69641a0764625f6e616d651a0b736368656d615f6e616d651a0a7461626c655f6e616d651a0d746f74616c5f636f6c756d6e731a0d746f74616c5f696e64657865731a0973746f72655f6964731a167265706c69636174696f6e5f73697a655f62797465731a0c746f74616c5f72616e6765731a15746f74616c5f6c6976655f646174615f62797465731a10746f74616c5f646174615f62797465731a0e706572635f6c6976655f646174611a116c6173745f7570646174655f6572726f721a0c6c6173745f757064617465641a0a7461626c655f747970651a0764657461696c73200120022003200420052006200720082009200a200b200c200d200e200f201020112800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89cc8a89","value":"030ab0040a0c6a6f625f70726f67726573731844200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422e0a086672616374696f6e10031a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a087265736f6c76656410041a0d080310001800300050a40d6000200130006800700078008001008801009801004805528c010a077072696d6172791001180122066a6f625f696422077772697474656e2a086672616374696f6e2a087265736f6c76656430013002400040014a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2013a0a077072696d61727910001a066a6f625f69641a077772697474656e1a086672616374696f6e1a087265736f6c76656420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89cd8a89","value":"030ab8040a146a6f625f70726f67726573735f686973746f72791845200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422e0a086672616374696f6e10031a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a087265736f6c76656410041a0d080310001800300050a40d6000200130006800700078008001008801009801004805528c010a077072696d6172791001180122066a6f625f696422077772697474656e2a086672616374696f6e2a087265736f6c76656430013002400040014a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2013a0a077072696d61727910001a066a6f625f69641a077772697474656e1a086672616374696f6e1a087265736f6c76656420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ce8a89","value":"030ade030a0a6a6f625f7374617475731846200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422b0a0673746174757310031a0c080710001800300050196000200030006800700078008001008801009801004804527e0a077072696d6172791001180122066a6f625f696422077772697474656e2a0673746174757330013002400040014a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012c0a077072696d61727910001a066a6f625f69641a077772697474656e1a067374617475732001200220032803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89cf8a89","value":"030aa0040a0b6a6f625f6d6573736167651847200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042290a046b696e6410031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076d65737361676510041a0c0807100018003000501960002000300068007000780080010088010098010048055289010a077072696d6172791001180122066a6f625f696422077772697474656e22046b696e642a076d6573736167653001300230034000400140004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201350a077072696d61727910001a066a6f625f69641a077772697474656e1a046b696e641a076d65737361676520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89d08a89","value":"030ab4060a1570726570617265645f7472616e73616374696f6e731848200128013a00422e0a09676c6f62616c5f696410011a0c0807100018003000501960002000300068007000780080010088010098010042340a0e7472616e73616374696f6e5f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042340a0f7472616e73616374696f6e5f6b657910031a0c0808100018003000501160002001300068007000780080010088010098010042430a08707265706172656410041a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210051a0c08071000180030005019600020003000680070007800800100880100980100422d0a08646174616261736510061a0c08071000180030005019600020003000680070007800800100880100980100422e0a0968657572697374696310071a0c08071000180030005019600020013000680070007800800100880100980100480852bd010a077072696d617279100118012209676c6f62616c5f69642a0e7472616e73616374696f6e5f69642a0f7472616e73616374696f6e5f6b65792a0870726570617265642a056f776e65722a0864617461626173652a09686575726973746963300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2016d0a077072696d61727910001a09676c6f62616c5f69641a0e7472616e73616374696f6e5f69641a0f7472616e73616374696f6e5f6b65791a0870726570617265641a056f776e65721a0864617461626173651a0968657572697374696320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} ,{"key":"8c"} ,{"key":"8d"} ,{"key":"8d89888a89","value":"031080808040188080808002220308c0702803500058007801"} @@ -213,72 +213,72 @@ system hash=8e0584706f674381cb24315b2254e0e8ba091a6246c670d4243fe9603ad4cb5e ,{"key":"d0"} ] -tenant hash=1fdb41445d6744a299fa74a47ca0246d66eecb1bfda15329afc1354c83404c1e +tenant hash=c4fae332fb8b5160f7e4f33bfff4d30064840a54f3fd7e2d750b8dff3a35452c ---- [{"key":""} ,{"key":"8b89898a89","value":"0312470a0673797374656d10011a250a0d0a0561646d696e1080101880100a0c0a04726f6f7410801018801012046e6f646518032200280140004a006a0a08d8843d1003180020127000"} -,{"key":"8b898b8a89","value":"030a94030a0a64657363726970746f721803200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a64657363726970746f7210021a0c08081000180030005011600020013000680070007800800100880100980100480352710a077072696d61727910011801220269642a0a64657363726970746f72300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201130a077072696d61727910001a02696420012800b201240a1066616d5f325f64657363726970746f7210021a0a64657363726970746f7220022802b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b898c8a89","value":"030acd050a0575736572731804200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042330a0e68617368656450617373776f726410021a0c0808100018003000501160002001300068007000780080010088010098010042320a066973526f6c6510031a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055290010a077072696d617279100118012208757365726e616d652a0e68617368656450617373776f72642a066973526f6c652a07757365725f6964300140004a10080010001a00200028003000380040005a007002700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a740a1175736572735f757365725f69645f696478100218012207757365725f69643004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201240a077072696d61727910001a08757365726e616d651a07757365725f6964200120042804b2012c0a1466616d5f325f68617368656450617373776f726410021a0e68617368656450617373776f726420022802b2011c0a0c66616d5f335f6973526f6c6510031a066973526f6c6520032803b80104c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b898d8a89","value":"030a83030a057a6f6e65731805200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422b0a06636f6e66696710021a0c080810001800300050116000200130006800700078008001008801009801004803526d0a077072696d61727910011801220269642a06636f6e666967300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201130a077072696d61727910001a02696420012800b2011c0a0c66616d5f325f636f6e66696710021a06636f6e66696720022802b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b898e8a89","value":"030ac2040a0873657474696e67731806200128013a0042290a046e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510021a0c0807100018003000501960002000300068007000780080010088010098010042440a0b6c6173745570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422e0a0976616c75655479706510041a0c080710001800300050196000200130006800700078008001008801009801004805528a010a077072696d6172791001180122046e616d652a0576616c75652a0b6c617374557064617465642a0976616c756554797065300140004a10080010001a00200028003000380040005a007002700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201590a2666616d5f305f6e616d655f76616c75655f6c617374557064617465645f76616c75655479706510001a046e616d651a0576616c75651a0b6c617374557064617465641a0976616c75655479706520012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b898f8a89","value":"030adf020a1164657363726970746f725f69645f7365711807200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011e0801100118ffffffffffffffff7f20012800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89908a89","value":"030afe060a0774656e616e74731808200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042310a0661637469766510021a0c08001000180030005010600020002a0474727565300168007000780080010088010098010042290a04696e666f10031a0c0808100018003000501160002001300068007000780080010088010098010042290a046e616d6510041a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a646174615f737461746510051a0c0801104018003000501460002001300068007000780080010088010098010042310a0c736572766963655f6d6f646510061a0c080110401800300050146000200130006800700078008001008801009801004807529b010a077072696d61727910011801220269642a066163746976652a04696e666f2a046e616d652a0a646174615f73746174652a0c736572766963655f6d6f6465300140004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a700a1074656e616e74735f6e616d655f6964781002180122046e616d653004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a80010a1874656e616e74735f736572766963655f6d6f64655f69647810031800220c736572766963655f6d6f64653006380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060046a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2014b0a077072696d61727910001a0269641a066163746976651a04696e666f1a046e616d651a0a646174615f73746174651a0c736572766963655f6d6f64652001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89918a89","value":"030aba030a0f726567696f6e5f6c6976656e6573731809200128013a0042300a0b637264625f726567696f6e10011a0c0808100018003000501160002000300068007000780080010088010098010042340a0e756e617661696c61626c655f617410021a0d080510001800300050da086000200130006800700078008001008801009801004803528b010a14726567696f6e5f6c6976656e6573735f706b657910011801220b637264625f726567696f6e2a0e756e617661696c61626c655f6174300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012e0a077072696d61727910001a0b637264625f726567696f6e1a0e756e617661696c61626c655f6174200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89938a89","value":"030a80050a056c65617365180b200128013a00422c0a07646573635f696410011a0c08011040180030005014600020003000680070007800800100880100980100422c0a0776657273696f6e10021a0c0801104018003000501460002000300068007000780080010088010098010042340a0f73716c5f696e7374616e63655f696410031a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a73657373696f6e5f696410041a0c0808100018003000501160002000300068007000780080010088010098010042300a0b637264625f726567696f6e10051a0c08081000180030005011600020003000680070007800800100880100980100480652a9010a077072696d61727910031801220b637264625f726567696f6e2207646573635f6964220776657273696f6e220a73657373696f6e5f69642a0f73716c5f696e7374616e63655f6964300530013002300440004000400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201530a077072696d61727910001a07646573635f69641a0776657273696f6e1a0f73716c5f696e7374616e63655f69641a0a73657373696f6e5f69641a0b637264625f726567696f6e200120022003200420052803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89948a89","value":"030a93060a086576656e746c6f67180c200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422e0a096576656e745479706510021a0c08071000180030005019600020003000680070007800800100880100980100422d0a08746172676574494410031a0c0801104018003000501460002000300068007000780080010088010098010042300a0b7265706f7274696e67494410041a0c0801104018003000501460002000300068007000780080010088010098010042290a04696e666f10051a0c0807100018003000501960002001300068007000780080010088010098010042380a08756e69717565494410061a0c08081000180030005011600020002a09757569645f763428293000680070007800800100880100980100480752a8010a077072696d61727910011801220974696d657374616d702208756e6971756549442a096576656e74547970652a0874617267657449442a0b7265706f7274696e6749442a04696e666f30013006400040004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120062800b201220a0f66616d5f325f6576656e745479706510021a096576656e745479706520022802b201200a0e66616d5f335f746172676574494410031a08746172676574494420032803b201260a1166616d5f345f7265706f7274696e67494410041a0b7265706f7274696e67494420042804b201180a0a66616d5f355f696e666f10051a04696e666f20052805b80106c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89958a89","value":"030af2060a0872616e67656c6f67180d200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422c0a0772616e6765494410021a0c08011040180030005014600020003000680070007800800100880100980100422c0a0773746f7265494410031a0c08011040180030005014600020003000680070007800800100880100980100422e0a096576656e745479706510041a0c0807100018003000501960002000300068007000780080010088010098010042310a0c6f7468657252616e6765494410051a0c0801104018003000501460002001300068007000780080010088010098010042290a04696e666f10061a0c08071000180030005019600020013000680070007800800100880100980100423d0a08756e69717565494410071a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100480852b3010a077072696d61727910011801220974696d657374616d702208756e6971756549442a0772616e676549442a0773746f726549442a096576656e74547970652a0c6f7468657252616e676549442a04696e666f30013007400040004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120072800b2011e0a0d66616d5f325f72616e6765494410021a0772616e6765494420022802b2011e0a0d66616d5f335f73746f7265494410031a0773746f7265494420032803b201220a0f66616d5f345f6576656e745479706510041a096576656e745479706520042804b201280a1266616d5f355f6f7468657252616e6765494410051a0c6f7468657252616e6765494420052805b201180a0a66616d5f365f696e666f10061a04696e666f20062806b80107c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89968a89","value":"030aea030a027569180e200128013a0042280a036b657910011a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510021a0c0808100018003000501160002001300068007000780080010088010098010042310a0b6c6173745570646174656410031a0d080510001800300050da086000200030006800700078008001008801009801004804527c0a077072696d6172791001180122036b65792a0576616c75652a0b6c61737455706461746564300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201140a077072696d61727910001a036b657920012800b2011a0a0b66616d5f325f76616c756510021a0576616c756520022802b201260a1166616d5f335f6c6173745570646174656410031a0b6c6173745570646174656420032803b80104c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89978a89","value":"030aac120a046a6f6273180f200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100422b0a0673746174757310021a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f64726f707065645f7061796c6f616410041a0c0808100018003000501160002001300168007000780080010088010098010042350a1064726f707065645f70726f677265737310051a0c0808100018003000501160002001300168007000780080010088010098010042340a0f637265617465645f62795f7479706510061a0c0807100018003000501960002001300068007000780080010088010098010042320a0d637265617465645f62795f696410071a0c0801104018003000501460002001300068007000780080010088010098010042350a10636c61696d5f73657373696f6e5f696410081a0c0808100018003000501160002001300068007000780080010088010098010042360a11636c61696d5f696e7374616e63655f696410091a0c08011040180030005014600020013000680070007800800100880100980100422d0a086e756d5f72756e73100a1a0c08011040180030005014600020013000680070007800800100880100980100422e0a086c6173745f72756e100b1a0d080510001800300050da08600020013000680070007800800100880100980100422d0a086a6f625f74797065100c1a0c08071000180030005019600020013000680070007800800100880100980100422a0a056f776e6572100d1a0c0807100018003000501960002001300068007000780080010088010098010042300a0b6465736372697074696f6e100e1a0c08071000180030005019600020013000680070007800800100880100980100422e0a096572726f725f6d7367100f1a0c08071000180030005019600020013000680070007800800100880100980100422e0a0866696e697368656410101a0d080910001800300050a009600020013000680070007800800100880100980100481152c1020a077072696d61727910011801220269642a067374617475732a07637265617465642a0f64726f707065645f7061796c6f61642a1064726f707065645f70726f67726573732a0f637265617465645f62795f747970652a0d637265617465645f62795f69642a10636c61696d5f73657373696f6e5f69642a11636c61696d5f696e7374616e63655f69642a086e756d5f72756e732a086c6173745f72756e2a086a6f625f747970652a056f776e65722a0b6465736372697074696f6e2a096572726f725f6d73672a0866696e6973686564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b700c700d700e700f70107a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a86010a176a6f62735f7374617475735f637265617465645f696478100218002206737461747573220763726561746564300230033801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aae010a266a6f62735f637265617465645f62795f747970655f637265617465645f62795f69645f69647810031800220f637265617465645f62795f74797065220d637265617465645f62795f69642a06737461747573300630073801400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ac6020a126a6f62735f72756e5f73746174735f696478100418002210636c61696d5f73657373696f6e5f696422067374617475732207637265617465642a086c6173745f72756e2a086e756d5f72756e732a11636c61696d5f696e7374616e63655f696430083002300338014000400040004a10080010001a00200028003000380040005a00700b700a70097a0408002000800100880100900103980100a20106080012001800a80100b20100ba01810173746174757320494e20282772756e6e696e67273a3a3a535452494e472c2027726576657274696e67273a3a3a535452494e472c202770656e64696e67273a3a3a535452494e472c202770617573652d726571756573746564273a3a3a535452494e472c202763616e63656c2d726571756573746564273a3a3a535452494e4729c00100c80100d00100e00100e90100000000000000005a750a116a6f62735f6a6f625f747970655f6964781005180022086a6f625f74797065300c380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201b4010a1f66616d5f305f69645f7374617475735f637265617465645f7061796c6f616410001a0269641a067374617475731a07637265617465641a0f64726f707065645f7061796c6f61641a0f637265617465645f62795f747970651a0d637265617465645f62795f69641a086a6f625f747970651a056f776e65721a0b6465736372697074696f6e1a096572726f725f6d73671a0866696e6973686564200120022003200420062007200c200d200e200f20102800b201220a0870726f677265737310011a1064726f707065645f70726f677265737320052805b2014c0a05636c61696d10021a10636c61696d5f73657373696f6e5f69641a11636c61696d5f696e7374616e63655f69641a086e756d5f72756e731a086c6173745f72756e20082009200a200b2800b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b899b8a89","value":"030a940c0a0c7765625f73657373696f6e731813200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042310a0c68617368656453656372657410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a08757365726e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042420a0963726561746564417410041a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422f0a0965787069726573417410051a0d080510001800300050da08600020003000680070007800800100880100980100422f0a097265766f6b6564417410061a0d080510001800300050da0860002001300068007000780080010088010098010042430a0a6c61737455736564417410071a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422e0a096175646974496e666f10081a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410091a0c080c100018003000501a600020003000680070007800800100880100980100480a52cc010a077072696d61727910011801220269642a0c6861736865645365637265742a08757365726e616d652a096372656174656441742a096578706972657341742a097265766f6b656441742a0a6c6173745573656441742a096175646974496e666f2a07757365725f6964300140004a10080010001a00200028003000380040005a00700270037004700570067007700870097a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f6578706972657341745f6964781002180022096578706972657341743005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f6372656174656441745f6964781003180022096372656174656441743004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f7265766f6b656441745f6964781004180022097265766f6b656441743006380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a81010a1b7765625f73657373696f6e735f6c6173745573656441745f69647810051800220a6c6173745573656441743007380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201c6010a5166616d5f305f69645f6861736865645365637265745f757365726e616d655f6372656174656441745f6578706972657341745f7265766f6b656441745f6c6173745573656441745f6175646974496e666f10001a0269641a0c6861736865645365637265741a08757365726e616d651a096372656174656441741a096578706972657341741a097265766f6b656441741a0a6c6173745573656441741a096175646974496e666f1a07757365725f69642001200220032004200520062007200820092800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b899c8a89","value":"030aaf0a0a107461626c655f737461746973746963731814200128013a00422c0a077461626c65494410011a0c0801104018003000501460002000300068007000780080010088010098010042400a0b737461746973746963494410021a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042290a046e616d6510031a0c08071000180030005019600020013000680070007800800100880100980100423f0a09636f6c756d6e49447310041a1d080f104018003000380150f8075a0c08011040180030005014600060002000300068007000780080010088010098010042420a0963726561746564417410051a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422d0a08726f77436f756e7410061a0c0801104018003000501460002000300068007000780080010088010098010042320a0d64697374696e6374436f756e7410071a0c08011040180030005014600020003000680070007800800100880100980100422e0a096e756c6c436f756e7410081a0c08011040180030005014600020003000680070007800800100880100980100422e0a09686973746f6772616d10091a0c0808100018003000501160002001300068007000780080010088010098010042360a0761766753697a65100a1a0c08011040180030005014600020002a08303a3a3a494e5438300068007000780080010088010098010042350a107061727469616c507265646963617465100b1a0c0807100018003000501960002001300068007000780080010088010098010042340a0f66756c6c5374617469737469634944100c1a0c08011040180030005014600020013000680070007800800100880100980100480d5284020a077072696d6172791001180122077461626c654944220b73746174697374696349442a046e616d652a09636f6c756d6e4944732a096372656174656441742a08726f77436f756e742a0d64697374696e6374436f756e742a096e756c6c436f756e742a09686973746f6772616d2a0761766753697a652a107061727469616c5072656469636174652a0f66756c6c537461746973746963494430013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b20188020a5d66616d5f305f7461626c6549445f73746174697374696349445f6e616d655f636f6c756d6e4944735f6372656174656441745f726f77436f756e745f64697374696e6374436f756e745f6e756c6c436f756e745f686973746f6772616d10001a077461626c6549441a0b73746174697374696349441a046e616d651a09636f6c756d6e4944731a096372656174656441741a08726f77436f756e741a0d64697374696e6374436f756e741a096e756c6c436f756e741a09686973746f6772616d1a0761766753697a651a107061727469616c5072656469636174651a0f66756c6c5374617469737469634944200120022003200420052006200720082009200a200b200c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b899d8a89","value":"030ae3040a096c6f636174696f6e731815200128013a0042300a0b6c6f63616c6974794b657910011a0c0807100018003000501960002000300068007000780080010088010098010042320a0d6c6f63616c69747956616c756510021a0c08071000180030005019600020003000680070007800800100880100980100422e0a086c6174697475646510031a0d0803100f1812300050a40d600020003000680070007800800100880100980100422f0a096c6f6e67697475646510041a0d0803100f1812300050a40d60002000300068007000780080010088010098010048055298010a077072696d61727910011801220b6c6f63616c6974794b6579220d6c6f63616c69747956616c75652a086c617469747564652a096c6f6e67697475646530013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201710a3266616d5f305f6c6f63616c6974794b65795f6c6f63616c69747956616c75655f6c617469747564655f6c6f6e67697475646510001a0b6c6f63616c6974794b65791a0d6c6f63616c69747956616c75651a086c617469747564651a096c6f6e67697475646520012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b899f8a89","value":"030aa20a0a0c726f6c655f6d656d626572731817200128013a0042290a04726f6c6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066d656d62657210021a0c08071000180030005019600020003000680070007800800100880100980100422c0a07697341646d696e10031a0c08001000180030005010600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a600020003000680070007800800100880100980100422e0a096d656d6265725f696410051a0c080c100018003000501a60002000300068007000780080010088010098010048065294010a077072696d617279100118012204726f6c6522066d656d6265722a07697341646d696e2a07726f6c655f69642a096d656d6265725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a750a15726f6c655f6d656d626572735f726f6c655f696478100218002204726f6c653001380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a790a17726f6c655f6d656d626572735f6d656d6265725f6964781003180022066d656d6265723002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7d0a18726f6c655f6d656d626572735f726f6c655f69645f696478100418002207726f6c655f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a81010a1a726f6c655f6d656d626572735f6d656d6265725f69645f6964781005180022096d656d6265725f696430053801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a96010a22726f6c655f6d656d626572735f726f6c655f69645f6d656d6265725f69645f6b6579100618012207726f6c655f696422096d656d6265725f69643004300538013802400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060076a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2011f0a077072696d61727910001a04726f6c651a066d656d626572200120022800b2011e0a0d66616d5f335f697341646d696e10031a07697341646d696e20032803b2011e0a0d66616d5f345f726f6c655f696410041a07726f6c655f696420042804b201220a0f66616d5f355f6d656d6265725f696410051a096d656d6265725f696420052805b80106c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a08a89","value":"030aae040a08636f6d6d656e74731818200128013a0042290a047479706510011a0c08011040180030005014600020003000680070007800800100880100980100422e0a096f626a6563745f696410021a0c08011040180030005014600020003000680070007800800100880100980100422b0a067375625f696410031a0c08011040180030005014600020003000680070007800800100880100980100422c0a07636f6d6d656e7410041a0c080710001800300050196000200030006800700078008001008801009801004805528b010a077072696d6172791001180122047479706522096f626a6563745f696422067375625f69642a07636f6d6d656e743001300230034000400040004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a330a0d0a0561646d696e10e00318e0030a0c0a067075626c6963102018000a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012c0a077072696d61727910001a04747970651a096f626a6563745f69641a067375625f69642001200220032800b2011e0a0d66616d5f345f636f6d6d656e7410041a07636f6d6d656e7420042804b80105c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a18a89","value":"030aa9060a1c7265706c69636174696f6e5f636f6e73747261696e745f73746174731819200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c0801104018003000501460002000300068007000780080010088010098010042290a047479706510031a0c08071000180030005019600020003000680070007800800100880100980100422b0a06636f6e66696710041a0c08071000180030005019600020003000680070007800800100880100980100422e0a097265706f72745f696410051a0c0801104018003000501460002000300068007000780080010088010098010042350a0f76696f6c6174696f6e5f737461727410061a0d080910001800300050a00960002001300068007000780080010088010098010042350a1076696f6c6174696e675f72616e67657310071a0c08011040180030005014600020003000680070007800800100880100980100480852c2010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f69642204747970652206636f6e6669672a097265706f72745f69642a0f76696f6c6174696f6e5f73746172742a1076696f6c6174696e675f72616e676573300130023003300440004000400040004a10080010001a00200028003000380040005a007005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2016c0a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a04747970651a06636f6e6669671a097265706f72745f69641a0f76696f6c6174696f6e5f73746172741a1076696f6c6174696e675f72616e67657320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a28a89","value":"030a92050a1f7265706c69636174696f6e5f637269746963616c5f6c6f63616c6974696573181a200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422d0a086c6f63616c69747910031a0c08071000180030005019600020003000680070007800800100880100980100422e0a097265706f72745f696410041a0c0801104018003000501460002000300068007000780080010088010098010042330a0e61745f7269736b5f72616e67657310051a0c08011040180030005014600020003000680070007800800100880100980100480652a5010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f696422086c6f63616c6974792a097265706f72745f69642a0e61745f7269736b5f72616e6765733001300230034000400040004a10080010001a00200028003000380040005a00700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201510a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a086c6f63616c6974791a097265706f72745f69641a0e61745f7269736b5f72616e676573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a38a89","value":"030a80070a117265706c69636174696f6e5f7374617473181b200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422e0a097265706f72745f696410031a0c0801104018003000501460002000300068007000780080010088010098010042310a0c746f74616c5f72616e67657310041a0c0801104018003000501460002000300068007000780080010088010098010042370a12756e617661696c61626c655f72616e67657310051a0c08011040180030005014600020003000680070007800800100880100980100423c0a17756e6465725f7265706c6963617465645f72616e67657310061a0c08011040180030005014600020003000680070007800800100880100980100423b0a166f7665725f7265706c6963617465645f72616e67657310071a0c08011040180030005014600020003000680070007800800100880100980100480852e0010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f69642a097265706f72745f69642a0c746f74616c5f72616e6765732a12756e617661696c61626c655f72616e6765732a17756e6465725f7265706c6963617465645f72616e6765732a166f7665725f7265706c6963617465645f72616e67657330013002400040004a10080010001a00200028003000380040005a00700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2018e010a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a097265706f72745f69641a0c746f74616c5f72616e6765731a12756e617661696c61626c655f72616e6765731a17756e6465725f7265706c6963617465645f72616e6765731a166f7665725f7265706c6963617465645f72616e67657320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a48a89","value":"030aff020a0c7265706f7274735f6d657461181c200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0967656e65726174656410021a0d080910001800300050a009600020003000680070007800800100880100980100480352700a077072696d61727910011801220269642a0967656e657261746564300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201200a077072696d61727910001a0269641a0967656e657261746564200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a68a89","value":"030a9e040a096e616d657370616365181e200128013a00422d0a08706172656e74494410011a0c0801104018003000501460002000300068007000780080010088010098010042330a0e706172656e74536368656d61494410021a0c0801104018003000501460002000300068007000780080010088010098010042290a046e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042270a02696410041a0c080110401800300050146000200130006800700078008001008801009801004805528d010a077072696d617279100118012208706172656e744944220e706172656e74536368656d61494422046e616d652a0269643001300230034000400040004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201330a077072696d61727910001a08706172656e7449441a0e706172656e74536368656d6149441a046e616d652001200220032800b201140a0866616d5f345f696410041a02696420042804b80105c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a78a89","value":"030aa8050a1170726f7465637465645f74735f6d657461181f200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422c0a0776657273696f6e10021a0c0801104018003000501460002000300068007000780080010088010098010042300a0b6e756d5f7265636f72647310031a0c08011040180030005014600020003000680070007800800100880100980100422e0a096e756d5f7370616e7310041a0c0801104018003000501460002000300068007000780080010088010098010042300a0b746f74616c5f627974657310051a0c08011040180030005014600020003000680070007800800100880100980100480652a0010a077072696d61727910011801220973696e676c65746f6e2a0776657273696f6e2a0b6e756d5f7265636f7264732a096e756d5f7370616e732a0b746f74616c5f6279746573300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201260a0973696e676c65746f6e120f636865636b5f73696e676c65746f6e18002801300038004002b201500a077072696d61727910001a0973696e676c65746f6e1a0776657273696f6e1a0b6e756d5f7265636f7264731a096e756d5f7370616e731a0b746f74616c5f6279746573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a88a89","value":"030a8c060a1470726f7465637465645f74735f7265636f7264731820200128013a0042280a02696410011a0d080e10001800300050861760002000300068007000780080010088010098010042280a02747310021a0d080310001800300050a40d600020003000680070007800800100880100980100422e0a096d6574615f7479706510031a0c0807100018003000501960002000300068007000780080010088010098010042290a046d65746110041a0c08081000180030005011600020013000680070007800800100880100980100422e0a096e756d5f7370616e7310051a0c08011040180030005014600020003000680070007800800100880100980100422a0a057370616e7310061a0c0808100018003000501160002000300068007000780080010088010098010042340a08766572696669656410071a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422b0a0674617267657410081a0c08081000180030005011600020013000680070007800800100880100980100480952aa010a077072696d61727910011801220269642a0274732a096d6574615f747970652a046d6574612a096e756d5f7370616e732a057370616e732a0876657269666965642a06746172676574300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2015a0a077072696d61727910001a0269641a0274731a096d6574615f747970651a046d6574611a096e756d5f7370616e731a057370616e731a0876657269666965641a06746172676574200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89a98a89","value":"030a84050a0c726f6c655f6f7074696f6e731821200128013a00422d0a08757365726e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066f7074696f6e10021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055289010a077072696d617279100118012208757365726e616d6522066f7074696f6e2a0576616c75652a07757365725f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a760a1175736572735f757365725f69645f696478100218002207757365725f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201370a077072696d61727910001a08757365726e616d651a066f7074696f6e1a0576616c75651a07757365725f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89aa8a89","value":"030ada030a1773746174656d656e745f62756e646c655f6368756e6b731822200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042300a0b6465736372697074696f6e10021a0c0807100018003000501960002001300068007000780080010088010098010042290a046461746110031a0c080810001800300050116000200030006800700078008001008801009801004804527a0a077072696d61727910011801220269642a0b6465736372697074696f6e2a0464617461300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012a0a077072696d61727910001a0269641a0b6465736372697074696f6e1a04646174612001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ab8a89","value":"030ae50c0a1e73746174656d656e745f646961676e6f73746963735f72657175657374731823200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042350a09636f6d706c6574656410021a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410031a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f646961676e6f73746963735f696410041a0c0801104018003000501460002001300068007000780080010088010098010042320a0c7265717565737465645f617410051a0d080910001800300050a00960002000300068007000780080010088010098010042410a156d696e5f657865637574696f6e5f6c6174656e637910061a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042300a0a657870697265735f617410071a0d080910001800300050a009600020013000680070007800800100880100980100423a0a1473616d706c696e675f70726f626162696c69747910081a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a09706c616e5f6769737410091a0c0807100018003000501960002001300068007000780080010088010098010042330a0e616e74695f706c616e5f67697374100a1a0c0800100018003000501060002001300068007000780080010088010098010042340a087265646163746564100b1a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100480c529f020a077072696d61727910011801220269642a09636f6d706c657465642a1573746174656d656e745f66696e6765727072696e742a1873746174656d656e745f646961676e6f73746963735f69642a0c7265717565737465645f61742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f676973742a087265646163746564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005afb010a0d636f6d706c657465645f696478100218002209636f6d706c65746564220269642a1573746174656d656e745f66696e6765727072696e742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f676973742a08726564616374656430023001400040004a10080010001a00200028003000380040005a0070037006700770087009700a700b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201620a3a73616d706c696e675f70726f626162696c697479204245545745454e20302e303a3a3a464c4f41543820414e4420312e303a3a3a464c4f415438121a636865636b5f73616d706c696e675f70726f626162696c69747918002808300038004002b201cf010a077072696d61727910001a0269641a09636f6d706c657465641a1573746174656d656e745f66696e6765727072696e741a1873746174656d656e745f646961676e6f73746963735f69641a0c7265717565737465645f61741a156d696e5f657865637574696f6e5f6c6174656e63791a0a657870697265735f61741a1473616d706c696e675f70726f626162696c6974791a09706c616e5f676973741a0e616e74695f706c616e5f676973741a087265646163746564200120022003200420052006200720082009200a200b2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ac8a89","value":"030abe060a1573746174656d656e745f646961676e6f73746963731824200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410021a0c08071000180030005019600020003000680070007800800100880100980100422e0a0973746174656d656e7410031a0c0807100018003000501960002000300068007000780080010088010098010042320a0c636f6c6c65637465645f617410041a0d080910001800300050a009600020003000680070007800800100880100980100422b0a05747261636510051a0d081210001800300050da1d60002001300068007000780080010088010098010042430a0d62756e646c655f6368756e6b7310061a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a056572726f7210071a0c08071000180030005019600020013000680070007800800100880100980100480852bc010a077072696d61727910011801220269642a1573746174656d656e745f66696e6765727072696e742a0973746174656d656e742a0c636f6c6c65637465645f61742a0574726163652a0d62756e646c655f6368756e6b732a056572726f72300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2016c0a077072696d61727910001a0269641a1573746174656d656e745f66696e6765727072696e741a0973746174656d656e741a0c636f6c6c65637465645f61741a0574726163651a0d62756e646c655f6368756e6b731a056572726f7220012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ad8a89","value":"030ad3090a0e7363686564756c65645f6a6f62731825200128013a0042400a0b7363686564756c655f696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042320a0d7363686564756c655f6e616d6510021a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210041a0c08071000180030005019600020003000680070007800800100880100980100422e0a086e6578745f72756e10051a0d080910001800300050a00960002001300068007000780080010088010098010042330a0e7363686564756c655f737461746510061a0c0808100018003000501160002001300068007000780080010088010098010042320a0d7363686564756c655f6578707210071a0c0807100018003000501960002001300068007000780080010088010098010042350a107363686564756c655f64657461696c7310081a0c0808100018003000501160002001300068007000780080010088010098010042320a0d6578656375746f725f7479706510091a0c0807100018003000501960002000300068007000780080010088010098010042330a0e657865637574696f6e5f61726773100a1a0c08081000180030005011600020003000680070007800800100880100980100480b52f7010a077072696d61727910011801220b7363686564756c655f69642a0d7363686564756c655f6e616d652a07637265617465642a056f776e65722a086e6578745f72756e2a0e7363686564756c655f73746174652a0d7363686564756c655f657870722a107363686564756c655f64657461696c732a0d6578656375746f725f747970652a0e657865637574696f6e5f61726773300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a700a0c6e6578745f72756e5f6964781002180022086e6578745f72756e3005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201380a05736368656410001a0b7363686564756c655f69641a086e6578745f72756e1a0e7363686564756c655f73746174652001200520062800b201780a056f7468657210011a0d7363686564756c655f6e616d651a07637265617465641a056f776e65721a0d7363686564756c655f657870721a107363686564756c655f64657461696c731a0d6578656375746f725f747970651a0e657865637574696f6e5f61726773200220032004200720082009200a2800b80102c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89af8a89","value":"030aec030a0b73716c6c6976656e6573731827200128013a00422f0a0a73657373696f6e5f696410011a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10021a0d080310001800300050a40d60002000300068007000780080010088010098010042300a0b637264625f726567696f6e10031a0c080810001800300050116000200030006800700078008001008801009801004804528a010a077072696d61727910021801220b637264625f726567696f6e220a73657373696f6e5f69642a0a65787069726174696f6e30033001400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201380a077072696d61727910001a0b637264625f726567696f6e1a0a73657373696f6e5f69641a0a65787069726174696f6e2003200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b08a89","value":"030ad9040a0a6d6967726174696f6e731828200128013a00422a0a056d616a6f7210011a0c08011040180030005014600020003000680070007800800100880100980100422a0a056d696e6f7210021a0c08011040180030005014600020003000680070007800800100880100980100422a0a05706174636810031a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e7465726e616c10041a0c0801104018003000501460002000300068007000780080010088010098010042320a0c636f6d706c657465645f617410051a0d080910001800300050a0096000200030006800700078008001008801009801004806529a010a077072696d6172791001180122056d616a6f7222056d696e6f72220570617463682208696e7465726e616c2a0c636f6d706c657465645f6174300130023003300440004000400040004a10080010001a00200028003000380040005a0070057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201440a077072696d61727910001a056d616a6f721a056d696e6f721a0570617463681a08696e7465726e616c1a0c636f6d706c657465645f6174200120022003200420052805b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b18a89","value":"030ac3030a0b6a6f696e5f746f6b656e731829200128013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422b0a0673656372657410021a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10031a0d080910001800300050a0096000200030006800700078008001008801009801004804527b0a077072696d61727910011801220269642a067365637265742a0a65787069726174696f6e300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012b0a077072696d61727910001a0269641a067365637265741a0a65787069726174696f6e2001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b28a89","value":"030a8f280a1473746174656d656e745f73746174697374696373182a200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410061a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10071a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110081a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310091a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e100a1a0d081210001800300050da1d600020003000680070007800800100880100980100429c020a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38100b1a0c080110201800300050176000200030015a8c016d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f69642c20706c616e5f686173682c207472616e73616374696f6e5f66696e6765727072696e745f696429292c20383a3a3a494e543829680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100c1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d3000680070007800800100880100980100426c0a0d696e64657865735f7573616765100d1a0d081210001800300050da1d6000200130005a3728737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27696e6465786573273a3a3a535452494e4768007000780080010188010098010042710a0f657865637574696f6e5f636f756e74100e1a0c080110401800300050146000200130005a3b2828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27636e74273a3a3a535452494e47293a3a494e5438680070007800800100880100980100428a010a0f736572766963655f6c6174656e6379100f1a0d080210401800300050bd056000200130005a53282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f4154386800700078008001008801009801004297010a0d6370755f73716c5f6e616e6f7310101a0d080210401800300050bd056000200130005a62282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e2763707553514c4e616e6f73273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100429c010a0f636f6e74656e74696f6e5f74696d6510111a0d080210401800300050bd056000200130005a65282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e27636f6e74656e74696f6e54696d65273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f41543868007000780080010088010098010042dc010a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d6510121a0d080210401800300050bd056000200130005a95012828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e3e27636e74273a3a3a535452494e47293a3a464c4f415438202a20282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100428a010a0b7039395f6c6174656e637910131a0d080210401800300050bd056000200130005a57282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e276c6174656e6379496e666f273a3a3a535452494e47292d3e27703939273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100481452c2050a077072696d617279100118012268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e732a0f657865637574696f6e5f636f756e742a0f736572766963655f6c6174656e63792a0d6370755f73716c5f6e616e6f732a0f636f6e74656e74696f6e5f74696d652a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d652a0b7039395f6c6174656e6379300b30013002300330043005300640004000400040004000400040004a10080010001a00200028003000380040005a00700770087009700a700c700e700f70107011701270137a0408002000800100880100900104980101a201c70108011268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f69642209706c616e5f68617368221a7472616e73616374696f6e5f66696e6765727072696e745f6964a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa7010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380b3801380438053806400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a89010a11696e64657865735f75736167655f69647810031800220d696e64657865735f7573616765300d380b38013802380338043805380640004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e00100e90100000000000000005acf010a13657865637574696f6e5f636f756e745f69647810041800220d616767726567617465645f747322086170705f6e616d65220f657865637574696f6e5f636f756e7430013005300e380b38023803380438064000400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ad1010a13736572766963655f6c6174656e63795f69647810051800220d616767726567617465645f747322086170705f6e616d65220f736572766963655f6c6174656e637930013005300f380b38023803380438064000400040014a10080010001a00200028003000380040005a00680f7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a116370755f73716c5f6e616e6f735f69647810061800220d616767726567617465645f747322086170705f6e616d65220d6370755f73716c5f6e616e6f73300130053010380b38023803380438064000400040014a10080010001a00200028003000380040005a0068107a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ad1010a13636f6e74656e74696f6e5f74696d655f69647810071800220d616767726567617465645f747322086170705f6e616d65220f636f6e74656e74696f6e5f74696d65300130053011380b38023803380438064000400040014a10080010001a00200028003000380040005a0068117a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005aef010a22746f74616c5f657374696d617465645f657865637574696f6e5f74696d655f69647810081800220d616767726567617465645f747322086170705f6e616d65221e746f74616c5f657374696d617465645f657865637574696f6e5f74696d65300130053012380b38023803380438064000400040014a10080010001a00200028003000380040005a0068127a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac9010a0f7039395f6c6174656e63795f69647810091800220d616767726567617465645f747322086170705f6e616d65220b7039395f6c6174656e6379300130053013380b38023803380438064000400040014a10080010001a00200028003000380040005a0068137a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e9010000000000000000600a6a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201b9020abc01637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e543829126e636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381800280b300038014002b201a4030a077072696d61727910001a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e731a0f657865637574696f6e5f636f756e741a0f736572766963655f6c6174656e63791a0d6370755f73716c5f6e616e6f731a0f636f6e74656e74696f6e5f74696d651a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d651a0b7039395f6c6174656e6379200b200120022003200420052006200720082009200a200c200e200f20102011201220132800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b38a89","value":"030a95200a167472616e73616374696f6e5f73746174697374696373182b200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410041a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10051a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110061a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310071a0d081210001800300050da1d60002000300068007000780080010088010098010042cf010a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3810081a0c080110201800300050176000200030015a656d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f696429292c20383a3a3a494e54382968007000780080010088010098010042710a0f657865637574696f6e5f636f756e7410091a0c080110401800300050146000200130005a3b2828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27636e74273a3a3a535452494e47293a3a494e5438680070007800800100880100980100428a010a0f736572766963655f6c6174656e6379100a1a0d080210401800300050bd056000200130005a53282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f4154386800700078008001008801009801004297010a0d6370755f73716c5f6e616e6f73100b1a0d080210401800300050bd056000200130005a62282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e2763707553514c4e616e6f73273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100429c010a0f636f6e74656e74696f6e5f74696d65100c1a0d080210401800300050bd056000200130005a65282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e27636f6e74656e74696f6e54696d65273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f41543868007000780080010088010098010042dc010a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d65100d1a0d080210401800300050bd056000200130005a95012828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e3e27636e74273a3a3a535452494e47293a3a464c4f415438202a20282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100428a010a0b7039395f6c6174656e6379100e1a0d080210401800300050bd056000200130005a57282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e276c6174656e6379496e666f273a3a3a535452494e47292d3e27703939273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100480f5280040a077072696d617279100118012243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a0f657865637574696f6e5f636f756e742a0f736572766963655f6c6174656e63792a0d6370755f73716c5f6e616e6f732a0f636f6e74656e74696f6e5f74696d652a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d652a0b7039395f6c6174656e637930083001300230033004400040004000400040004a10080010001a00200028003000380040005a007005700670077009700a700b700c700d700e7a0408002000800100880100900104980101a2017b08011243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f6964a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a85010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f69643002380838013803380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005acb010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f747322086170705f6e616d65220f657865637574696f6e5f636f756e743001300330093808380238044000400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a13736572766963655f6c6174656e63795f69647810041800220d616767726567617465645f747322086170705f6e616d65220f736572766963655f6c6174656e637930013003300a3808380238044000400040014a10080010001a00200028003000380040005a00680a7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac9010a116370755f73716c5f6e616e6f735f69647810051800220d616767726567617465645f747322086170705f6e616d65220d6370755f73716c5f6e616e6f7330013003300b3808380238044000400040014a10080010001a00200028003000380040005a00680b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a13636f6e74656e74696f6e5f74696d655f69647810061800220d616767726567617465645f747322086170705f6e616d65220f636f6e74656e74696f6e5f74696d6530013003300c3808380238044000400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005aeb010a22746f74616c5f657374696d617465645f657865637574696f6e5f74696d655f69647810071800220d616767726567617465645f747322086170705f6e616d65221e746f74616c5f657374696d617465645f657865637574696f6e5f74696d6530013003300d3808380238044000400040014a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac5010a0f7039395f6c6174656e63795f69647810081800220d616767726567617465645f747322086170705f6e616d65220b7039395f6c6174656e637930013003300e3808380238044000400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201ef010a9701637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e5438291249636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3818002808300038014002b201b3020a077072696d61727910001a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a0f657865637574696f6e5f636f756e741a0f736572766963655f6c6174656e63791a0d6370755f73716c5f6e616e6f731a0f636f6e74656e74696f6e5f74696d651a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d651a0b7039395f6c6174656e6379200820012002200320042005200620072009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b48a89","value":"030af3050a1664617461626173655f726f6c655f73657474696e6773182c200128013a0042300a0b64617461626173655f696410011a0c080c100018003000501a600020003000680070007800800100880100980100422e0a09726f6c655f6e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100423e0a0873657474696e677310031a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055292010a077072696d61727910011801220b64617461626173655f69642209726f6c655f6e616d652a0873657474696e67732a07726f6c655f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005aae010a2e64617461626173655f726f6c655f73657474696e67735f64617461626173655f69645f726f6c655f69645f6b657910021801220b64617461626173655f69642207726f6c655f69642a0873657474696e6773300130043802400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201400a077072696d61727910001a0b64617461626173655f69641a09726f6c655f6e616d651a0873657474696e67731a07726f6c655f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b58a89","value":"030ac00b0a0c74656e616e745f7573616765182d200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042300a0b696e7374616e63655f696410021a0c0801104018003000501460002000300068007000780080010088010098010042350a106e6578745f696e7374616e63655f696410031a0c0801104018003000501460002000300068007000780080010088010098010042310a0b6c6173745f75706461746510041a0d080510001800300050da0860002000300068007000780080010088010098010042340a0e72755f62757273745f6c696d697410051a0d080210401800300050bd0560002001300068007000780080010088010098010042340a0e72755f726566696c6c5f7261746510061a0d080210401800300050bd0560002001300068007000780080010088010098010042300a0a72755f63757272656e7410071a0d080210401800300050bd0560002001300068007000780080010088010098010042370a1163757272656e745f73686172655f73756d10081a0d080210401800300050bd0560002001300068007000780080010088010098010042360a11746f74616c5f636f6e73756d7074696f6e10091a0c0808100018003000501160002001300068007000780080010088010098010042330a0e696e7374616e63655f6c65617365100a1a0c0808100018003000501160002001300068007000780080010088010098010042310a0c696e7374616e63655f736571100b1a0c0801104018003000501460002001300068007000780080010088010098010042350a0f696e7374616e63655f736861726573100c1a0d080210401800300050bd0560002001300068007000780080010088010098010042320a0d63757272656e745f7261746573100d1a0c08081000180030005011600020013000680070007800800100880100980100422f0a0a6e6578745f7261746573100e1a0c08081000180030005011600020013000680070007800800100880100980100480f52ce020a077072696d61727910011801220974656e616e745f6964220b696e7374616e63655f69642a106e6578745f696e7374616e63655f69642a0b6c6173745f7570646174652a0e72755f62757273745f6c696d69742a0e72755f726566696c6c5f726174652a0a72755f63757272656e742a1163757272656e745f73686172655f73756d2a11746f74616c5f636f6e73756d7074696f6e2a0e696e7374616e63655f6c656173652a0c696e7374616e63655f7365712a0f696e7374616e63655f7368617265732a0d63757272656e745f72617465732a0a6e6578745f726174657330013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c700d700e7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201fc010a077072696d61727910001a0974656e616e745f69641a0b696e7374616e63655f69641a106e6578745f696e7374616e63655f69641a0b6c6173745f7570646174651a0e72755f62757273745f6c696d69741a0e72755f726566696c6c5f726174651a0a72755f63757272656e741a1163757272656e745f73686172655f73756d1a11746f74616c5f636f6e73756d7074696f6e1a0e696e7374616e63655f6c656173651a0c696e7374616e63655f7365711a0f696e7374616e63655f7368617265731a0d63757272656e745f72617465731a0a6e6578745f7261746573200120022003200420052006200720082009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b68a89","value":"030ac8060a0d73716c5f696e7374616e636573182e200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046164647210021a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410031a0c08081000180030005011600020013000680070007800800100880100980100422e0a086c6f63616c69747910041a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0873716c5f6164647210051a0c0807100018003000501960002001300068007000780080010088010098010042300a0b637264625f726567696f6e10061a0c0808100018003000501160002000300068007000780080010088010098010042330a0e62696e6172795f76657273696f6e10071a0c0807100018003000501960002001300068007000780080010088010098010042300a0b69735f647261696e696e6710081a0c08001000180030005010600020013000680070007800800100880100980100480952c3010a077072696d61727910021801220b637264625f726567696f6e220269642a04616464722a0a73657373696f6e5f69642a086c6f63616c6974792a0873716c5f616464722a0e62696e6172795f76657273696f6e2a0b69735f647261696e696e6730063001400040004a10080010001a00200028003000380040005a007002700370047005700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201710a077072696d61727910001a0269641a04616464721a0a73657373696f6e5f69641a086c6f63616c6974791a0873716c5f616464721a0b637264625f726567696f6e1a0e62696e6172795f76657273696f6e1a0b69735f647261696e696e67200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b78a89","value":"030a87040a137370616e5f636f6e66696775726174696f6e73182f200128013a00422e0a0973746172745f6b657910011a0c08081000180030005011600020003000680070007800800100880100980100422c0a07656e645f6b657910021a0c08081000180030005011600020003000680070007800800100880100980100422b0a06636f6e66696710031a0c080810001800300050116000200030006800700078008001008801009801004804527f0a077072696d61727910011801220973746172745f6b65792a07656e645f6b65792a06636f6e666967300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a2012f0a1373746172745f6b6579203c20656e645f6b6579120c636865636b5f626f756e6473180028012802300038004002b2012f0a077072696d61727910001a0973746172745f6b65791a07656e645f6b65791a06636f6e6669672001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89b88a89","value":"030ad9020a0b726f6c655f69645f7365711830200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a250a0d0a0561646d696e10a00618a0060a0c0a04726f6f7410a00618a00612046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011a0801106418ffffffff0720642800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ba8a89","value":"030af0050a0f74656e616e745f73657474696e67731832200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c0807100018003000501960002000300068007000780080010088010098010042450a0c6c6173745f7570646174656410041a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422f0a0a76616c75655f7479706510051a0c08071000180030005019600020003000680070007800800100880100980100422b0a06726561736f6e10061a0c08071000180030005019600020013000680070007800800100880100980100480752a5010a077072696d61727910011801220974656e616e745f696422046e616d652a0576616c75652a0c6c6173745f757064617465642a0a76616c75655f747970652a06726561736f6e30013002400040004a10080010001a00200028003000380040005a0070037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b20185010a3966616d5f305f74656e616e745f69645f6e616d655f76616c75655f6c6173745f757064617465645f76616c75655f747970655f726561736f6e10001a0974656e616e745f69641a046e616d651a0576616c75651a0c6c6173745f757064617465641a0a76616c75655f747970651a06726561736f6e2001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89bb8a89","value":"030abe030a0a7370616e5f636f756e741833200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422f0a0a7370616e5f636f756e7410021a0c08011040180030005014600020003000680070007800800100880100980100480352780a077072696d61727910011801220973696e676c65746f6e2a0a7370616e5f636f756e74300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201210a0973696e676c65746f6e120a73696e676c655f726f7718002801300038004002b201280a077072696d61727910001a0973696e676c65746f6e1a0a7370616e5f636f756e74200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89bc8a89","value":"030adf070a0a70726976696c656765731834200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042290a047061746810021a0c0807100018003000501960002000300068007000780080010088010098010042400a0a70726976696c6567657310031a1d080f100018003000380750f1075a0c08071000180030005019600060002000300068007000780080010088010098010042430a0d6772616e745f6f7074696f6e7310041a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07757365725f696410051a0c080c100018003000501a6000200030006800700078008001008801009801004806529d010a077072696d617279100118012208757365726e616d652204706174682a0a70726976696c656765732a0d6772616e745f6f7074696f6e732a07757365725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00103e00100e90100000000000000005aa7010a1b70726976696c656765735f706174685f757365725f69645f6b6579100218012204706174682207757365725f69642a0a70726976696c656765732a0d6772616e745f6f7074696f6e73300230053801400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa7010a1c70726976696c656765735f706174685f757365726e616d655f6b6579100318012204706174682208757365726e616d652a0a70726976696c656765732a0d6772616e745f6f7074696f6e7330023001400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2014b0a077072696d61727910001a08757365726e616d651a04706174681a0a70726976696c656765731a0d6772616e745f6f7074696f6e731a07757365725f6964200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880304a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89bd8a89","value":"030ada060a1465787465726e616c5f636f6e6e656374696f6e731835200128013a0042340a0f636f6e6e656374696f6e5f6e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042400a077570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f636f6e6e656374696f6e5f7479706510041a0c0807100018003000501960002000300068007000780080010088010098010042370a12636f6e6e656374696f6e5f64657461696c7310051a0c08081000180030005011600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852c4010a077072696d61727910011801220f636f6e6e656374696f6e5f6e616d652a07637265617465642a07757064617465642a0f636f6e6e656374696f6e5f747970652a12636f6e6e656374696f6e5f64657461696c732a056f776e65722a086f776e65725f6964300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201740a077072696d61727910001a0f636f6e6e656374696f6e5f6e616d651a07637265617465641a07757064617465641a0f636f6e6e656374696f6e5f747970651a12636f6e6e656374696f6e5f64657461696c731a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89be8a89","value":"030aa0040a086a6f625f696e666f1836200128013a00422b0a066a6f625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e666f5f6b657910021a0c0807100018003000501960002000300068007000780080010088010098010042420a077772697474656e10031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a0576616c756510041a0c080810001800300050116000200130006800700078008001008801009801004805528b010a077072696d6172791001180122066a6f625f69642208696e666f5f6b657922077772697474656e2a0576616c75653001300230034000400040014a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201370a077072696d61727910001a066a6f625f69641a08696e666f5f6b65791a077772697474656e1a0576616c756520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89bf8a89","value":"030a9c040a167370616e5f73746174735f756e697175655f6b6579731837200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422e0a096b65795f627974657310021a0c08081000180030005011600020013000680070007800800100880100980100480352700a077072696d61727910011801220269642a096b65795f6279746573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a7e0a19756e697175655f6b6579735f6b65795f62797465735f6964781002180122096b65795f62797465733002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201200a077072696d61727910001a0269641a096b65795f6279746573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c08a89","value":"030aff050a127370616e5f73746174735f6275636b6574731838200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422f0a0973616d706c655f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042320a0c73746172745f6b65795f696410031a0d080e10001800300050861760002000300068007000780080010088010098010042300a0a656e645f6b65795f696410041a0d080e100018003000508617600020003000680070007800800100880100980100422d0a08726571756573747310051a0c080110401800300050146000200030006800700078008001008801009801004806529a010a077072696d61727910011801220269642a0973616d706c655f69642a0c73746172745f6b65795f69642a0a656e645f6b65795f69642a087265717565737473300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7a0a156275636b6574735f73616d706c655f69645f69647810021800220973616d706c655f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2014a0a077072696d61727910001a0269641a0973616d706c655f69641a0c73746172745f6b65795f69641a0a656e645f6b65795f69641a087265717565737473200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c18a89","value":"030ab2040a127370616e5f73746174735f73616d706c65731839200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f757569642829300068007000780080010088010098010042440a0b73616d706c655f74696d6510021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100480352720a077072696d61727910011801220269642a0b73616d706c655f74696d65300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a7e0a1773616d706c65735f73616d706c655f74696d655f69647810021801220b73616d706c655f74696d653002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201220a077072696d61727910001a0269641a0b73616d706c655f74696d65200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c28a89","value":"030aa6030a1c7370616e5f73746174735f74656e616e745f626f756e646172696573183a200128013a00422e0a0974656e616e745f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a626f756e64617269657310021a0c08081000180030005011600020003000680070007800800100880100980100480352780a077072696d61727910011801220974656e616e745f69642a0a626f756e646172696573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201280a077072696d61727910001a0974656e616e745f69641a0a626f756e646172696573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c38a89","value":"030aae060a0d7461736b5f7061796c6f616473183b200128013a0042270a02696410011a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210031a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010042300a0b6d696e5f76657273696f6e10051a0c0807100018003000501960002000300068007000780080010088010098010042300a0b6465736372697074696f6e10061a0c0807100018003000501960002001300068007000780080010088010098010042290a047479706510071a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510081a0c08081000180030005011600020003000680070007800800100880100980100480952b2010a077072696d61727910011801220269642a07637265617465642a056f776e65722a086f776e65725f69642a0b6d696e5f76657273696f6e2a0b6465736372697074696f6e2a04747970652a0576616c7565300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201620a077072696d61727910001a0269641a07637265617465641a056f776e65721a086f776e65725f69641a0b6d696e5f76657273696f6e1a0b6465736372697074696f6e1a04747970651a0576616c7565200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c48a89","value":"030aff050a0c74656e616e745f7461736b73183c200128013a00422e0a0974656e616e745f696410011a0c08011040180030005014600020003000680070007800800100880100980100422b0a0669737375657210021a0c08071000180030005019600020003000680070007800800100880100980100422c0a077461736b5f696410031a0c0801104018003000501460002000300068007000780080010088010098010042420a076372656174656410041a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422f0a0a7061796c6f61645f696410051a0c08071000180030005019600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852b1010a077072696d61727910011801220974656e616e745f6964220669737375657222077461736b5f69642a07637265617465642a0a7061796c6f61645f69642a056f776e65722a086f776e65725f69643001300230034000400040004a10080010001a00200028003000380040005a0070047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2015d0a077072696d61727910001a0974656e616e745f69641a066973737565721a077461736b5f69641a07637265617465641a0a7061796c6f61645f69641a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c58a89","value":"030af1180a1273746174656d656e745f6163746976697479183d200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c0807100018003000501960002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10061a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110071a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310081a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e10091a0d081210001800300050da1d600020003000680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100a1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d300068007000780080010088010098010042340a0f657865637574696f6e5f636f756e74100b1a0c08011040180030005014600020003000680070007800800100880100980100423d0a17657865637574696f6e5f746f74616c5f7365636f6e6473100c1a0d080210401800300050bd0560002000300068007000780080010088010098010042450a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e6473100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e6473100e1a0d080210401800300050bd0560002000300068007000780080010088010098010042370a116370755f73716c5f6176675f6e616e6f73100f1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f6176675f7365636f6e647310101a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f7039395f7365636f6e647310111a0d080210401800300050bd05600020003000680070007800800100880100980100481252cd030a077072696d61727910011801220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d652a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e732a0f657865637574696f6e5f636f756e742a17657865637574696f6e5f746f74616c5f7365636f6e64732a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64732a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64732a116370755f73716c5f6176675f6e616e6f732a1b736572766963655f6c6174656e63795f6176675f7365636f6e64732a1b736572766963655f6c6174656e63795f7039395f7365636f6e647330013002300330043005400040004000400040004a10080010001a00200028003000380040005a007006700770087009700a700b700c700d700e700f701070117a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa0010a1266696e6765727072696e745f69645f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380138043805400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a97010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f7473220f657865637574696f6e5f636f756e743001300b3802380338043805400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aa9010a1b657865637574696f6e5f746f74616c5f7365636f6e64735f69647810041800220d616767726567617465645f74732217657865637574696f6e5f746f74616c5f7365636f6e64733001300c3802380338043805400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64735f69647810051800220d616767726567617465645f7473221b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64733001300e3802380338043805400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a9d010a156370755f73716c5f6176675f6e616e6f735f69647810061800220d616767726567617465645f747322116370755f73716c5f6176675f6e616e6f733001300f3802380338043805400040014a10080010001a00200028003000380040005a00680f7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f736572766963655f6c6174656e63795f6176675f7365636f6e64735f69647810071800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f6176675f7365636f6e6473300130103802380338043805400040014a10080010001a00200028003000380040005a0068107a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f736572766963655f6c6174656e63795f7039395f7365636f6e64735f69647810081800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f7039395f7365636f6e6473300130113802380338043805400040014a10080010001a00200028003000380040005a0068117a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201f5020a077072696d61727910001a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e731a0f657865637574696f6e5f636f756e741a17657865637574696f6e5f746f74616c5f7365636f6e64731a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64731a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64731a116370755f73716c5f6176675f6e616e6f731a1b736572766963655f6c6174656e63795f6176675f7365636f6e64731a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473200120022003200420052006200720082009200a200b200c200d200e200f201020112800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c68a89","value":"030adc150a147472616e73616374696f6e5f6163746976697479183e200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10041a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110051a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310061a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a05717565727910071a0c0807100018003000501960002000300068007000780080010088010098010042340a0f657865637574696f6e5f636f756e7410081a0c08011040180030005014600020003000680070007800800100880100980100423d0a17657865637574696f6e5f746f74616c5f7365636f6e647310091a0d080210401800300050bd0560002000300068007000780080010088010098010042450a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e6473100a1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e6473100b1a0d080210401800300050bd0560002000300068007000780080010088010098010042370a116370755f73716c5f6176675f6e616e6f73100c1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f6176675f7365636f6e6473100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473100e1a0d080210401800300050bd05600020003000680070007800800100880100980100480f5286030a077072696d61727910011801220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d652a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a0571756572792a0f657865637574696f6e5f636f756e742a17657865637574696f6e5f746f74616c5f7365636f6e64732a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64732a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64732a116370755f73716c5f6176675f6e616e6f732a1b736572766963655f6c6174656e63795f6176675f7365636f6e64732a1b736572766963655f6c6174656e63795f7039395f7365636f6e64733001300230034000400040004a10080010001a00200028003000380040005a00700470057006700770087009700a700b700c700d700e7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7e0a1266696e6765727072696e745f69645f69647810021800220e66696e6765727072696e745f696430023801380340004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a93010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f7473220f657865637574696f6e5f636f756e743001300838023803400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aa5010a1b657865637574696f6e5f746f74616c5f7365636f6e64735f69647810041800220d616767726567617465645f74732217657865637574696f6e5f746f74616c5f7365636f6e64733001300938023803400040014a10080010001a00200028003000380040005a0068097a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64735f69647810051800220d616767726567617465645f7473221b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64733001300b38023803400040014a10080010001a00200028003000380040005a00680b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a99010a156370755f73716c5f6176675f6e616e6f735f69647810061800220d616767726567617465645f747322116370755f73716c5f6176675f6e616e6f733001300c38023803400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f736572766963655f6c6174656e63795f6176675f7365636f6e64735f69647810071800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f6176675f7365636f6e64733001300d38023803400040014a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f736572766963655f6c6174656e63795f7039395f7365636f6e64735f69647810081800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f7039395f7365636f6e64733001300e38023803400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201b2020a077072696d61727910001a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a0571756572791a0f657865637574696f6e5f636f756e741a17657865637574696f6e5f746f74616c5f7365636f6e64731a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64731a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64731a116370755f73716c5f6176675f6e616e6f731a1b736572766963655f6c6174656e63795f6176675f7365636f6e64731a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473200120022003200420052006200720082009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c78a89","value":"030adb020a0d74656e616e745f69645f736571183f200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011e0801100118ffffffffffffffff7f20012800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c88a89","value":"030ac50a0a0f6d7663635f737461746973746963731840200128013a0042450a0a637265617465645f617410011a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042300a0b64617461626173655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422d0a087461626c655f696410031a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e6465785f696410041a0c0801104018003000501460002000300068007000780080010088010098010042300a0a7374617469737469637310051a0d081210001800300050da1d60002000300068007000780080010088010098010042ab010a3f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313610061a0c080110201800300050176000200030015a456d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328637265617465645f61742929292c2031363a3a3a494e543829680070007800800101880100980100480752e4020a146d7663635f737461746973746963735f706b657910011801223f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f3136220a637265617465645f6174220b64617461626173655f696422087461626c655f69642208696e6465785f69642a0a7374617469737469637330063001300230033004400040004000400040004a10080010001a00200028003000380040005a0070057a0408002000800100880100900104980101a201720801123f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f31361810220a637265617465645f6174220b64617461626173655f69642208696e6465785f696422087461626c655f6964a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201bd020ae901637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291245636865636b5f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313618002806300038014002b201500a077072696d61727910001a0a637265617465645f61741a0b64617461626173655f69641a087461626c655f69641a08696e6465785f69641a0a73746174697374696373200120022003200420052805b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89c98a89","value":"030abb170a1e7472616e73616374696f6e5f657865637574696f6e5f696e7369676874731841200128013a0042340a0e7472616e73616374696f6e5f696410011a0d080e100018003000508617600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410021a0c0808100018003000501160002000300068007000780080010088010098010042320a0d71756572795f73756d6d61727910031a0c0807100018003000501960002001300068007000780080010088010098010042310a0c696d706c696369745f74786e10041a0c08001000180030005010600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410051a0c0807100018003000501960002000300068007000780080010088010098010042300a0a73746172745f74696d6510061a0d080910001800300050a009600020013000680070007800800100880100980100422e0a08656e645f74696d6510071a0d080910001800300050a009600020013000680070007800800100880100980100422e0a09757365725f6e616d6510081a0c08071000180030005019600020013000680070007800800100880100980100422d0a086170705f6e616d6510091a0c0807100018003000501960002001300068007000780080010088010098010042320a0d757365725f7072696f72697479100a1a0c08071000180030005019600020013000680070007800800100880100980100422c0a0772657472696573100b1a0c0801104018003000501460002001300068007000780080010088010098010042360a116c6173745f72657472795f726561736f6e100c1a0c08071000180030005019600020013000680070007800800100880100980100423e0a0870726f626c656d73100d1a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100423c0a06636175736573100e1a1d080f104018003000380150f8075a0c08011040180030005014600060002001300068007000780080010088010098010042480a1273746d745f657865637574696f6e5f696473100f1a1d080f100018003000380750f1075a0c08071000180030005019600060002001300068007000780080010088010098010042320a0d6370755f73716c5f6e616e6f7310101a0c0801104018003000501460002001300068007000780080010088010098010042340a0f6c6173745f6572726f725f636f646510111a0c08071000180030005019600020013000680070007800800100880100980100422b0a0673746174757310121a0c08011040180030005014600020013000680070007800800100880100980100423b0a0f636f6e74656e74696f6e5f74696d6510131a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042350a0f636f6e74656e74696f6e5f696e666f10141a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0764657461696c7310151a0d081210001800300050da1d60002001300068007000780080010088010098010042420a076372656174656410161a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042a0010a2a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313610171a0c080110201800300050176000200030015a4f6d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328656e645f74696d652c2073746172745f74696d652929292c2031363a3a3a494e543829680070007800800101880100980100481852b6030a077072696d61727910011801220e7472616e73616374696f6e5f69642a1a7472616e73616374696f6e5f66696e6765727072696e745f69642a0d71756572795f73756d6d6172792a0c696d706c696369745f74786e2a0a73657373696f6e5f69642a0a73746172745f74696d652a08656e645f74696d652a09757365725f6e616d652a086170705f6e616d652a0d757365725f7072696f726974792a07726574726965732a116c6173745f72657472795f726561736f6e2a0870726f626c656d732a066361757365732a1273746d745f657865637574696f6e5f6964732a0d6370755f73716c5f6e616e6f732a0f6c6173745f6572726f725f636f64652a067374617475732a0f636f6e74656e74696f6e5f74696d652a0f636f6e74656e74696f6e5f696e666f2a0764657461696c732a0763726561746564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b700c700d700e700f70107011701270137014701570167a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a94010a1e7472616e73616374696f6e5f66696e6765727072696e745f69645f69647810021800221a7472616e73616374696f6e5f66696e6765727072696e745f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005af2010a0e74696d655f72616e67655f69647810031800222a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136220a73746172745f74696d652208656e645f74696d6530173006300738014000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a201460801122a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618102208656e645f74696d65220a73746172745f74696d65a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20193020ad401637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291230636865636b5f637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618002817300038014002b201e6020a077072696d61727910001a0e7472616e73616374696f6e5f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a0d71756572795f73756d6d6172791a0c696d706c696369745f74786e1a0a73657373696f6e5f69641a0a73746172745f74696d651a08656e645f74696d651a09757365725f6e616d651a086170705f6e616d651a0d757365725f7072696f726974791a07726574726965731a116c6173745f72657472795f726561736f6e1a0870726f626c656d731a066361757365731a1273746d745f657865637574696f6e5f6964731a0d6370755f73716c5f6e616e6f731a0f6c6173745f6572726f725f636f64651a067374617475731a0f636f6e74656e74696f6e5f74696d651a0f636f6e74656e74696f6e5f696e666f1a0764657461696c731a0763726561746564200120022003200420052006200720082009200a200b200c200d200e200f20102011201220132014201520162800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ca8a89","value":"030a861e0a1c73746174656d656e745f657865637574696f6e5f696e7369676874731842200128013a00422f0a0a73657373696f6e5f696410011a0c0807100018003000501960002000300068007000780080010088010098010042340a0e7472616e73616374696f6e5f696410021a0d080e100018003000508617600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c0808100018003000501160002000300068007000780080010088010098010042310a0c73746174656d656e745f696410041a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f66696e6765727072696e745f696410051a0c08081000180030005011600020003000680070007800800100880100980100422c0a0770726f626c656d10061a0c08011040180030005014600020013000680070007800800100880100980100423c0a0663617573657310071a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a05717565727910081a0c08071000180030005019600020013000680070007800800100880100980100422b0a0673746174757310091a0c0801104018003000501460002001300068007000780080010088010098010042300a0a73746172745f74696d65100a1a0d080910001800300050a009600020013000680070007800800100880100980100422e0a08656e645f74696d65100b1a0d080910001800300050a009600020013000680070007800800100880100980100422e0a0966756c6c5f7363616e100c1a0c08001000180030005010600020013000680070007800800100880100980100422e0a09757365725f6e616d65100d1a0c08071000180030005019600020013000680070007800800100880100980100422d0a086170705f6e616d65100e1a0c0807100018003000501960002001300068007000780080010088010098010042320a0d757365725f7072696f72697479100f1a0c0807100018003000501960002001300068007000780080010088010098010042320a0d64617461626173655f6e616d6510101a0c08071000180030005019600020013000680070007800800100880100980100422e0a09706c616e5f6769737410111a0c08071000180030005019600020013000680070007800800100880100980100422c0a077265747269657310121a0c0801104018003000501460002001300068007000780080010088010098010042360a116c6173745f72657472795f726561736f6e10131a0c0807100018003000501960002001300068007000780080010088010098010042480a12657865637574696f6e5f6e6f64655f69647310141a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100424b0a15696e6465785f7265636f6d6d656e646174696f6e7310151a1d080f100018003000380750f1075a0c08071000180030005019600060002001300068007000780080010088010098010042310a0c696d706c696369745f74786e10161a0c0800100018003000501060002001300068007000780080010088010098010042320a0d6370755f73716c5f6e616e6f7310171a0c08011040180030005014600020013000680070007800800100880100980100422f0a0a6572726f725f636f646510181a0c08071000180030005019600020013000680070007800800100880100980100423b0a0f636f6e74656e74696f6e5f74696d6510191a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042350a0f636f6e74656e74696f6e5f696e666f101a1a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0764657461696c73101b1a0d081210001800300050da1d60002001300068007000780080010088010098010042420a0763726561746564101c1a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042a0010a2a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136101d1a0c080110201800300050176000200030015a4f6d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328656e645f74696d652c2073746172745f74696d652929292c2031363a3a3a494e543829680070007800800101880100980100481e529a040a077072696d61727910011801220c73746174656d656e745f6964220e7472616e73616374696f6e5f69642a0a73657373696f6e5f69642a1a7472616e73616374696f6e5f66696e6765727072696e745f69642a1873746174656d656e745f66696e6765727072696e745f69642a0770726f626c656d2a066361757365732a0571756572792a067374617475732a0a73746172745f74696d652a08656e645f74696d652a0966756c6c5f7363616e2a09757365725f6e616d652a086170705f6e616d652a0d757365725f7072696f726974792a0d64617461626173655f6e616d652a09706c616e5f676973742a07726574726965732a116c6173745f72657472795f726561736f6e2a12657865637574696f6e5f6e6f64655f6964732a15696e6465785f7265636f6d6d656e646174696f6e732a0c696d706c696369745f74786e2a0d6370755f73716c5f6e616e6f732a0a6572726f725f636f64652a0f636f6e74656e74696f6e5f74696d652a0f636f6e74656e74696f6e5f696e666f2a0764657461696c732a076372656174656430043002400040004a10080010001a00200028003000380040005a007001700370057006700770087009700a700b700c700d700e700f7010701170127013701470157016701770187019701a701b701c7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7c0a127472616e73616374696f6e5f69645f69647810021800220e7472616e73616374696f6e5f69643002380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab4010a1e7472616e73616374696f6e5f66696e6765727072696e745f69645f69647810031800221a7472616e73616374696f6e5f66696e6765727072696e745f6964220a73746172745f74696d652208656e645f74696d653003300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab0010a1c73746174656d656e745f66696e6765727072696e745f69645f69647810041800221873746174656d656e745f66696e6765727072696e745f6964220a73746172745f74696d652208656e645f74696d653005300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005af4010a0e74696d655f72616e67655f69647810051800222a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136220a73746172745f74696d652208656e645f74696d65301d300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a201460801122a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618102208656e645f74696d65220a73746172745f74696d65a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20193020ad401637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291230636865636b5f637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f31361800281d300038014002b201c8030a077072696d61727910001a0a73657373696f6e5f69641a0e7472616e73616374696f6e5f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a0c73746174656d656e745f69641a1873746174656d656e745f66696e6765727072696e745f69641a0770726f626c656d1a066361757365731a0571756572791a067374617475731a0a73746172745f74696d651a08656e645f74696d651a0966756c6c5f7363616e1a09757365725f6e616d651a086170705f6e616d651a0d757365725f7072696f726974791a0d64617461626173655f6e616d651a09706c616e5f676973741a07726574726965731a116c6173745f72657472795f726561736f6e1a12657865637574696f6e5f6e6f64655f6964731a15696e6465785f7265636f6d6d656e646174696f6e731a0c696d706c696369745f74786e1a0d6370755f73716c5f6e616e6f731a0a6572726f725f636f64651a0f636f6e74656e74696f6e5f74696d651a0f636f6e74656e74696f6e5f696e666f1a0764657461696c731a0763726561746564200120022003200420052006200720082009200a200b200c200d200e200f2010201120122013201420152016201720182019201a201b201c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89cb8a89","value":"030ad31c0a0e7461626c655f6d657461646174611843200128013a00422a0a0564625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a087461626c655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422c0a0764625f6e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042300a0b736368656d615f6e616d6510041a0c08071000180030005019600020003000680070007800800100880100980100422f0a0a7461626c655f6e616d6510051a0c0807100018003000501960002000300068007000780080010088010098010042320a0d746f74616c5f636f6c756d6e7310061a0c0801104018003000501460002000300068007000780080010088010098010042320a0d746f74616c5f696e646578657310071a0c08011040180030005014600020003000680070007800800100880100980100423f0a0973746f72655f69647310081a1d080f104018003000380150f8075a0c080110401800300050146000600020003000680070007800800100880100980100423b0a167265706c69636174696f6e5f73697a655f627974657310091a0c0801104018003000501460002000300068007000780080010088010098010042310a0c746f74616c5f72616e676573100a1a0c08011040180030005014600020003000680070007800800100880100980100423a0a15746f74616c5f6c6976655f646174615f6279746573100b1a0c0801104018003000501460002000300068007000780080010088010098010042350a10746f74616c5f646174615f6279746573100c1a0c0801104018003000501460002000300068007000780080010088010098010042340a0e706572635f6c6976655f64617461100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042360a116c6173745f7570646174655f6572726f72100e1a0c0807100018003000501960002001300068007000780080010088010098010042470a0c6c6173745f75706461746564100f1a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422f0a0a7461626c655f7479706510101a0c08071000180030005019600020003000680070007800800100880100980100422d0a0764657461696c7310111a0d081210001800300050da1d60002000300068007000780080010088010098010042a4010a2c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313610121a0c080110201800300050176000200030015a516d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f6279746573287461626c655f69642c206c6173745f757064617465642929292c2031363a3a3a494e543829680070007800800101880100980100481352f2020a077072696d61727910011801220564625f696422087461626c655f69642a0764625f6e616d652a0b736368656d615f6e616d652a0a7461626c655f6e616d652a0d746f74616c5f636f6c756d6e732a0d746f74616c5f696e64657865732a0973746f72655f6964732a167265706c69636174696f6e5f73697a655f62797465732a0c746f74616c5f72616e6765732a15746f74616c5f6c6976655f646174615f62797465732a10746f74616c5f646174615f62797465732a0e706572635f6c6976655f646174612a116c6173745f7570646174655f6572726f722a0c6c6173745f757064617465642a0a7461626c655f747970652a0764657461696c7330013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c700d700e700f701070117a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa3010a237265706c69636174696f6e5f73697a655f62797465735f7461626c655f69645f6964781002180022167265706c69636174696f6e5f73697a655f627974657322087461626c655f6964300930023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a8f010a19746f74616c5f72616e6765735f7461626c655f69645f69647810031800220c746f74616c5f72616e67657322087461626c655f6964300a30023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a91010a1a746f74616c5f636f6c756d6e735f7461626c655f69645f69647810041800220d746f74616c5f636f6c756d6e7322087461626c655f6964300630023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a91010a1a746f74616c5f696e64657865735f7461626c655f69645f69647810051800220d746f74616c5f696e646578657322087461626c655f6964300730023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a8f010a15706572635f6c6976655f646174615f69645f69647810061800220e706572635f6c6976655f6461746122087461626c655f6964300d30023801400140004a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005afc010a106c6173745f757064617465645f69647810071800222c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f3136220c6c6173745f7570646174656422087461626c655f69643012300f300238014000400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a2014a0801122c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f31361810220c6c6173745f7570646174656422087461626c655f6964a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a730a0b64625f6e616d655f67696e10081800220764625f6e616d6530033801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a790a0e7461626c655f6e616d655f67696e10091800220a7461626c655f6e616d6530053801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a7b0a0f736368656d615f6e616d655f67696e100a1800220b736368656d615f6e616d6530043801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a770a0d73746f72655f6964735f67696e100b1800220973746f72655f69647330083801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e00100e9010000000000000000600c6a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20197020ad601637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291232636865636b5f637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313618002812300038014002b201a0020a077072696d61727910001a0564625f69641a087461626c655f69641a0764625f6e616d651a0b736368656d615f6e616d651a0a7461626c655f6e616d651a0d746f74616c5f636f6c756d6e731a0d746f74616c5f696e64657865731a0973746f72655f6964731a167265706c69636174696f6e5f73697a655f62797465731a0c746f74616c5f72616e6765731a15746f74616c5f6c6976655f646174615f62797465731a10746f74616c5f646174615f62797465731a0e706572635f6c6976655f646174611a116c6173745f7570646174655f6572726f721a0c6c6173745f757064617465641a0a7461626c655f747970651a0764657461696c73200120022003200420052006200720082009200a200b200c200d200e200f201020112800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89cc8a89","value":"030aad040a0c6a6f625f70726f67726573731844200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422e0a086672616374696f6e10031a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a087265736f6c76656410041a0d080310001800300050a40d6000200130006800700078008001008801009801004805528c010a077072696d6172791001180122066a6f625f696422077772697474656e2a086672616374696f6e2a087265736f6c76656430013002400040014a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2013a0a077072696d61727910001a066a6f625f69641a077772697474656e1a086672616374696f6e1a087265736f6c76656420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89cd8a89","value":"030ab5040a146a6f625f70726f67726573735f686973746f72791845200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422e0a086672616374696f6e10031a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a087265736f6c76656410041a0d080310001800300050a40d6000200130006800700078008001008801009801004805528c010a077072696d6172791001180122066a6f625f696422077772697474656e2a086672616374696f6e2a087265736f6c76656430013002400040014a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2013a0a077072696d61727910001a066a6f625f69641a077772697474656e1a086672616374696f6e1a087265736f6c76656420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89ce8a89","value":"030adb030a0a6a6f625f7374617475731846200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422b0a0673746174757310031a0c080710001800300050196000200030006800700078008001008801009801004804527e0a077072696d6172791001180122066a6f625f696422077772697474656e2a0673746174757330013002400040014a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012c0a077072696d61727910001a066a6f625f69641a077772697474656e1a067374617475732001200220032803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89cf8a89","value":"030a9d040a0b6a6f625f6d6573736167651847200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042290a046b696e6410031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076d65737361676510041a0c0807100018003000501960002000300068007000780080010088010098010048055289010a077072696d6172791001180122066a6f625f696422077772697474656e22046b696e642a076d6573736167653001300230034000400140004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201350a077072696d61727910001a066a6f625f69641a077772697474656e1a046b696e641a076d65737361676520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} -,{"key":"8b89d08a89","value":"030ab1060a1570726570617265645f7472616e73616374696f6e731848200128013a00422e0a09676c6f62616c5f696410011a0c0807100018003000501960002000300068007000780080010088010098010042340a0e7472616e73616374696f6e5f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042340a0f7472616e73616374696f6e5f6b657910031a0c0808100018003000501160002001300068007000780080010088010098010042430a08707265706172656410041a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210051a0c08071000180030005019600020003000680070007800800100880100980100422d0a08646174616261736510061a0c08071000180030005019600020003000680070007800800100880100980100422e0a0968657572697374696310071a0c08071000180030005019600020013000680070007800800100880100980100480852bd010a077072696d617279100118012209676c6f62616c5f69642a0e7472616e73616374696f6e5f69642a0f7472616e73616374696f6e5f6b65792a0870726570617265642a056f776e65722a0864617461626173652a09686575726973746963300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2016d0a077072696d61727910001a09676c6f62616c5f69641a0e7472616e73616374696f6e5f69641a0f7472616e73616374696f6e5f6b65791a0870726570617265641a056f776e65721a0864617461626173651a0968657572697374696320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400"} +,{"key":"8b898b8a89","value":"030a97030a0a64657363726970746f721803200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a64657363726970746f7210021a0c08081000180030005011600020013000680070007800800100880100980100480352710a077072696d61727910011801220269642a0a64657363726970746f72300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201130a077072696d61727910001a02696420012800b201240a1066616d5f325f64657363726970746f7210021a0a64657363726970746f7220022802b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b898c8a89","value":"030ad0050a0575736572731804200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042330a0e68617368656450617373776f726410021a0c0808100018003000501160002001300068007000780080010088010098010042320a066973526f6c6510031a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055290010a077072696d617279100118012208757365726e616d652a0e68617368656450617373776f72642a066973526f6c652a07757365725f6964300140004a10080010001a00200028003000380040005a007002700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a740a1175736572735f757365725f69645f696478100218012207757365725f69643004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201240a077072696d61727910001a08757365726e616d651a07757365725f6964200120042804b2012c0a1466616d5f325f68617368656450617373776f726410021a0e68617368656450617373776f726420022802b2011c0a0c66616d5f335f6973526f6c6510031a066973526f6c6520032803b80104c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b898d8a89","value":"030a86030a057a6f6e65731805200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422b0a06636f6e66696710021a0c080810001800300050116000200130006800700078008001008801009801004803526d0a077072696d61727910011801220269642a06636f6e666967300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201130a077072696d61727910001a02696420012800b2011c0a0c66616d5f325f636f6e66696710021a06636f6e66696720022802b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b898e8a89","value":"030ac5040a0873657474696e67731806200128013a0042290a046e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510021a0c0807100018003000501960002000300068007000780080010088010098010042440a0b6c6173745570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422e0a0976616c75655479706510041a0c080710001800300050196000200130006800700078008001008801009801004805528a010a077072696d6172791001180122046e616d652a0576616c75652a0b6c617374557064617465642a0976616c756554797065300140004a10080010001a00200028003000380040005a007002700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201590a2666616d5f305f6e616d655f76616c75655f6c617374557064617465645f76616c75655479706510001a046e616d651a0576616c75651a0b6c617374557064617465641a0976616c75655479706520012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b898f8a89","value":"030ae2020a1164657363726970746f725f69645f7365711807200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011e0801100118ffffffffffffffff7f20012800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89908a89","value":"030a81070a0774656e616e74731808200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042310a0661637469766510021a0c08001000180030005010600020002a0474727565300168007000780080010088010098010042290a04696e666f10031a0c0808100018003000501160002001300068007000780080010088010098010042290a046e616d6510041a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a646174615f737461746510051a0c0801104018003000501460002001300068007000780080010088010098010042310a0c736572766963655f6d6f646510061a0c080110401800300050146000200130006800700078008001008801009801004807529b010a077072696d61727910011801220269642a066163746976652a04696e666f2a046e616d652a0a646174615f73746174652a0c736572766963655f6d6f6465300140004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a700a1074656e616e74735f6e616d655f6964781002180122046e616d653004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a80010a1874656e616e74735f736572766963655f6d6f64655f69647810031800220c736572766963655f6d6f64653006380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060046a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2014b0a077072696d61727910001a0269641a066163746976651a04696e666f1a046e616d651a0a646174615f73746174651a0c736572766963655f6d6f64652001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89918a89","value":"030abd030a0f726567696f6e5f6c6976656e6573731809200128013a0042300a0b637264625f726567696f6e10011a0c0808100018003000501160002000300068007000780080010088010098010042340a0e756e617661696c61626c655f617410021a0d080510001800300050da086000200130006800700078008001008801009801004803528b010a14726567696f6e5f6c6976656e6573735f706b657910011801220b637264625f726567696f6e2a0e756e617661696c61626c655f6174300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012e0a077072696d61727910001a0b637264625f726567696f6e1a0e756e617661696c61626c655f6174200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89938a89","value":"030a83050a056c65617365180b200128013a00422c0a07646573635f696410011a0c08011040180030005014600020003000680070007800800100880100980100422c0a0776657273696f6e10021a0c0801104018003000501460002000300068007000780080010088010098010042340a0f73716c5f696e7374616e63655f696410031a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a73657373696f6e5f696410041a0c0808100018003000501160002000300068007000780080010088010098010042300a0b637264625f726567696f6e10051a0c08081000180030005011600020003000680070007800800100880100980100480652a9010a077072696d61727910031801220b637264625f726567696f6e2207646573635f6964220776657273696f6e220a73657373696f6e5f69642a0f73716c5f696e7374616e63655f6964300530013002300440004000400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201530a077072696d61727910001a07646573635f69641a0776657273696f6e1a0f73716c5f696e7374616e63655f69641a0a73657373696f6e5f69641a0b637264625f726567696f6e200120022003200420052803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89948a89","value":"030a96060a086576656e746c6f67180c200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422e0a096576656e745479706510021a0c08071000180030005019600020003000680070007800800100880100980100422d0a08746172676574494410031a0c0801104018003000501460002000300068007000780080010088010098010042300a0b7265706f7274696e67494410041a0c0801104018003000501460002000300068007000780080010088010098010042290a04696e666f10051a0c0807100018003000501960002001300068007000780080010088010098010042380a08756e69717565494410061a0c08081000180030005011600020002a09757569645f763428293000680070007800800100880100980100480752a8010a077072696d61727910011801220974696d657374616d702208756e6971756549442a096576656e74547970652a0874617267657449442a0b7265706f7274696e6749442a04696e666f30013006400040004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120062800b201220a0f66616d5f325f6576656e745479706510021a096576656e745479706520022802b201200a0e66616d5f335f746172676574494410031a08746172676574494420032803b201260a1166616d5f345f7265706f7274696e67494410041a0b7265706f7274696e67494420042804b201180a0a66616d5f355f696e666f10051a04696e666f20052805b80106c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89958a89","value":"030af5060a0872616e67656c6f67180d200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422c0a0772616e6765494410021a0c08011040180030005014600020003000680070007800800100880100980100422c0a0773746f7265494410031a0c08011040180030005014600020003000680070007800800100880100980100422e0a096576656e745479706510041a0c0807100018003000501960002000300068007000780080010088010098010042310a0c6f7468657252616e6765494410051a0c0801104018003000501460002001300068007000780080010088010098010042290a04696e666f10061a0c08071000180030005019600020013000680070007800800100880100980100423d0a08756e69717565494410071a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100480852b3010a077072696d61727910011801220974696d657374616d702208756e6971756549442a0772616e676549442a0773746f726549442a096576656e74547970652a0c6f7468657252616e676549442a04696e666f30013007400040004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120072800b2011e0a0d66616d5f325f72616e6765494410021a0772616e6765494420022802b2011e0a0d66616d5f335f73746f7265494410031a0773746f7265494420032803b201220a0f66616d5f345f6576656e745479706510041a096576656e745479706520042804b201280a1266616d5f355f6f7468657252616e6765494410051a0c6f7468657252616e6765494420052805b201180a0a66616d5f365f696e666f10061a04696e666f20062806b80107c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89968a89","value":"030aed030a027569180e200128013a0042280a036b657910011a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510021a0c0808100018003000501160002001300068007000780080010088010098010042310a0b6c6173745570646174656410031a0d080510001800300050da086000200030006800700078008001008801009801004804527c0a077072696d6172791001180122036b65792a0576616c75652a0b6c61737455706461746564300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201140a077072696d61727910001a036b657920012800b2011a0a0b66616d5f325f76616c756510021a0576616c756520022802b201260a1166616d5f335f6c6173745570646174656410031a0b6c6173745570646174656420032803b80104c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89978a89","value":"030aaf120a046a6f6273180f200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100422b0a0673746174757310021a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f64726f707065645f7061796c6f616410041a0c0808100018003000501160002001300168007000780080010088010098010042350a1064726f707065645f70726f677265737310051a0c0808100018003000501160002001300168007000780080010088010098010042340a0f637265617465645f62795f7479706510061a0c0807100018003000501960002001300068007000780080010088010098010042320a0d637265617465645f62795f696410071a0c0801104018003000501460002001300068007000780080010088010098010042350a10636c61696d5f73657373696f6e5f696410081a0c0808100018003000501160002001300068007000780080010088010098010042360a11636c61696d5f696e7374616e63655f696410091a0c08011040180030005014600020013000680070007800800100880100980100422d0a086e756d5f72756e73100a1a0c08011040180030005014600020013000680070007800800100880100980100422e0a086c6173745f72756e100b1a0d080510001800300050da08600020013000680070007800800100880100980100422d0a086a6f625f74797065100c1a0c08071000180030005019600020013000680070007800800100880100980100422a0a056f776e6572100d1a0c0807100018003000501960002001300068007000780080010088010098010042300a0b6465736372697074696f6e100e1a0c08071000180030005019600020013000680070007800800100880100980100422e0a096572726f725f6d7367100f1a0c08071000180030005019600020013000680070007800800100880100980100422e0a0866696e697368656410101a0d080910001800300050a009600020013000680070007800800100880100980100481152c1020a077072696d61727910011801220269642a067374617475732a07637265617465642a0f64726f707065645f7061796c6f61642a1064726f707065645f70726f67726573732a0f637265617465645f62795f747970652a0d637265617465645f62795f69642a10636c61696d5f73657373696f6e5f69642a11636c61696d5f696e7374616e63655f69642a086e756d5f72756e732a086c6173745f72756e2a086a6f625f747970652a056f776e65722a0b6465736372697074696f6e2a096572726f725f6d73672a0866696e6973686564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b700c700d700e700f70107a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a86010a176a6f62735f7374617475735f637265617465645f696478100218002206737461747573220763726561746564300230033801400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aae010a266a6f62735f637265617465645f62795f747970655f637265617465645f62795f69645f69647810031800220f637265617465645f62795f74797065220d637265617465645f62795f69642a06737461747573300630073801400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ac6020a126a6f62735f72756e5f73746174735f696478100418002210636c61696d5f73657373696f6e5f696422067374617475732207637265617465642a086c6173745f72756e2a086e756d5f72756e732a11636c61696d5f696e7374616e63655f696430083002300338014000400040004a10080010001a00200028003000380040005a00700b700a70097a0408002000800100880100900103980100a20106080012001800a80100b20100ba01810173746174757320494e20282772756e6e696e67273a3a3a535452494e472c2027726576657274696e67273a3a3a535452494e472c202770656e64696e67273a3a3a535452494e472c202770617573652d726571756573746564273a3a3a535452494e472c202763616e63656c2d726571756573746564273a3a3a535452494e4729c00100c80100d00100e00100e90100000000000000005a750a116a6f62735f6a6f625f747970655f6964781005180022086a6f625f74797065300c380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201b4010a1f66616d5f305f69645f7374617475735f637265617465645f7061796c6f616410001a0269641a067374617475731a07637265617465641a0f64726f707065645f7061796c6f61641a0f637265617465645f62795f747970651a0d637265617465645f62795f69641a086a6f625f747970651a056f776e65721a0b6465736372697074696f6e1a096572726f725f6d73671a0866696e6973686564200120022003200420062007200c200d200e200f20102800b201220a0870726f677265737310011a1064726f707065645f70726f677265737320052805b2014c0a05636c61696d10021a10636c61696d5f73657373696f6e5f69641a11636c61696d5f696e7374616e63655f69641a086e756d5f72756e731a086c6173745f72756e20082009200a200b2800b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b899b8a89","value":"030a970c0a0c7765625f73657373696f6e731813200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042310a0c68617368656453656372657410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a08757365726e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042420a0963726561746564417410041a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422f0a0965787069726573417410051a0d080510001800300050da08600020003000680070007800800100880100980100422f0a097265766f6b6564417410061a0d080510001800300050da0860002001300068007000780080010088010098010042430a0a6c61737455736564417410071a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422e0a096175646974496e666f10081a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410091a0c080c100018003000501a600020003000680070007800800100880100980100480a52cc010a077072696d61727910011801220269642a0c6861736865645365637265742a08757365726e616d652a096372656174656441742a096578706972657341742a097265766f6b656441742a0a6c6173745573656441742a096175646974496e666f2a07757365725f6964300140004a10080010001a00200028003000380040005a00700270037004700570067007700870097a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f6578706972657341745f6964781002180022096578706972657341743005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f6372656174656441745f6964781003180022096372656174656441743004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7f0a1a7765625f73657373696f6e735f7265766f6b656441745f6964781004180022097265766f6b656441743006380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a81010a1b7765625f73657373696f6e735f6c6173745573656441745f69647810051800220a6c6173745573656441743007380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201c6010a5166616d5f305f69645f6861736865645365637265745f757365726e616d655f6372656174656441745f6578706972657341745f7265766f6b656441745f6c6173745573656441745f6175646974496e666f10001a0269641a0c6861736865645365637265741a08757365726e616d651a096372656174656441741a096578706972657341741a097265766f6b656441741a0a6c6173745573656441741a096175646974496e666f1a07757365725f69642001200220032004200520062007200820092800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b899c8a89","value":"030ab20a0a107461626c655f737461746973746963731814200128013a00422c0a077461626c65494410011a0c0801104018003000501460002000300068007000780080010088010098010042400a0b737461746973746963494410021a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042290a046e616d6510031a0c08071000180030005019600020013000680070007800800100880100980100423f0a09636f6c756d6e49447310041a1d080f104018003000380150f8075a0c08011040180030005014600060002000300068007000780080010088010098010042420a0963726561746564417410051a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422d0a08726f77436f756e7410061a0c0801104018003000501460002000300068007000780080010088010098010042320a0d64697374696e6374436f756e7410071a0c08011040180030005014600020003000680070007800800100880100980100422e0a096e756c6c436f756e7410081a0c08011040180030005014600020003000680070007800800100880100980100422e0a09686973746f6772616d10091a0c0808100018003000501160002001300068007000780080010088010098010042360a0761766753697a65100a1a0c08011040180030005014600020002a08303a3a3a494e5438300068007000780080010088010098010042350a107061727469616c507265646963617465100b1a0c0807100018003000501960002001300068007000780080010088010098010042340a0f66756c6c5374617469737469634944100c1a0c08011040180030005014600020013000680070007800800100880100980100480d5284020a077072696d6172791001180122077461626c654944220b73746174697374696349442a046e616d652a09636f6c756d6e4944732a096372656174656441742a08726f77436f756e742a0d64697374696e6374436f756e742a096e756c6c436f756e742a09686973746f6772616d2a0761766753697a652a107061727469616c5072656469636174652a0f66756c6c537461746973746963494430013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b20188020a5d66616d5f305f7461626c6549445f73746174697374696349445f6e616d655f636f6c756d6e4944735f6372656174656441745f726f77436f756e745f64697374696e6374436f756e745f6e756c6c436f756e745f686973746f6772616d10001a077461626c6549441a0b73746174697374696349441a046e616d651a09636f6c756d6e4944731a096372656174656441741a08726f77436f756e741a0d64697374696e6374436f756e741a096e756c6c436f756e741a09686973746f6772616d1a0761766753697a651a107061727469616c5072656469636174651a0f66756c6c5374617469737469634944200120022003200420052006200720082009200a200b200c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b899d8a89","value":"030ae6040a096c6f636174696f6e731815200128013a0042300a0b6c6f63616c6974794b657910011a0c0807100018003000501960002000300068007000780080010088010098010042320a0d6c6f63616c69747956616c756510021a0c08071000180030005019600020003000680070007800800100880100980100422e0a086c6174697475646510031a0d0803100f1812300050a40d600020003000680070007800800100880100980100422f0a096c6f6e67697475646510041a0d0803100f1812300050a40d60002000300068007000780080010088010098010048055298010a077072696d61727910011801220b6c6f63616c6974794b6579220d6c6f63616c69747956616c75652a086c617469747564652a096c6f6e67697475646530013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201710a3266616d5f305f6c6f63616c6974794b65795f6c6f63616c69747956616c75655f6c617469747564655f6c6f6e67697475646510001a0b6c6f63616c6974794b65791a0d6c6f63616c69747956616c75651a086c617469747564651a096c6f6e67697475646520012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b899f8a89","value":"030aa50a0a0c726f6c655f6d656d626572731817200128013a0042290a04726f6c6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066d656d62657210021a0c08071000180030005019600020003000680070007800800100880100980100422c0a07697341646d696e10031a0c08001000180030005010600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a600020003000680070007800800100880100980100422e0a096d656d6265725f696410051a0c080c100018003000501a60002000300068007000780080010088010098010048065294010a077072696d617279100118012204726f6c6522066d656d6265722a07697341646d696e2a07726f6c655f69642a096d656d6265725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a750a15726f6c655f6d656d626572735f726f6c655f696478100218002204726f6c653001380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a790a17726f6c655f6d656d626572735f6d656d6265725f6964781003180022066d656d6265723002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a7d0a18726f6c655f6d656d626572735f726f6c655f69645f696478100418002207726f6c655f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a81010a1a726f6c655f6d656d626572735f6d656d6265725f69645f6964781005180022096d656d6265725f696430053801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a96010a22726f6c655f6d656d626572735f726f6c655f69645f6d656d6265725f69645f6b6579100618012207726f6c655f696422096d656d6265725f69643004300538013802400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060076a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2011f0a077072696d61727910001a04726f6c651a066d656d626572200120022800b2011e0a0d66616d5f335f697341646d696e10031a07697341646d696e20032803b2011e0a0d66616d5f345f726f6c655f696410041a07726f6c655f696420042804b201220a0f66616d5f355f6d656d6265725f696410051a096d656d6265725f696420052805b80106c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a08a89","value":"030ab1040a08636f6d6d656e74731818200128013a0042290a047479706510011a0c08011040180030005014600020003000680070007800800100880100980100422e0a096f626a6563745f696410021a0c08011040180030005014600020003000680070007800800100880100980100422b0a067375625f696410031a0c08011040180030005014600020003000680070007800800100880100980100422c0a07636f6d6d656e7410041a0c080710001800300050196000200030006800700078008001008801009801004805528b010a077072696d6172791001180122047479706522096f626a6563745f696422067375625f69642a07636f6d6d656e743001300230034000400040004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a330a0d0a0561646d696e10e00318e0030a0c0a067075626c6963102018000a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012c0a077072696d61727910001a04747970651a096f626a6563745f69641a067375625f69642001200220032800b2011e0a0d66616d5f345f636f6d6d656e7410041a07636f6d6d656e7420042804b80105c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a18a89","value":"030aac060a1c7265706c69636174696f6e5f636f6e73747261696e745f73746174731819200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c0801104018003000501460002000300068007000780080010088010098010042290a047479706510031a0c08071000180030005019600020003000680070007800800100880100980100422b0a06636f6e66696710041a0c08071000180030005019600020003000680070007800800100880100980100422e0a097265706f72745f696410051a0c0801104018003000501460002000300068007000780080010088010098010042350a0f76696f6c6174696f6e5f737461727410061a0d080910001800300050a00960002001300068007000780080010088010098010042350a1076696f6c6174696e675f72616e67657310071a0c08011040180030005014600020003000680070007800800100880100980100480852c2010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f69642204747970652206636f6e6669672a097265706f72745f69642a0f76696f6c6174696f6e5f73746172742a1076696f6c6174696e675f72616e676573300130023003300440004000400040004a10080010001a00200028003000380040005a007005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2016c0a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a04747970651a06636f6e6669671a097265706f72745f69641a0f76696f6c6174696f6e5f73746172741a1076696f6c6174696e675f72616e67657320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a28a89","value":"030a95050a1f7265706c69636174696f6e5f637269746963616c5f6c6f63616c6974696573181a200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422d0a086c6f63616c69747910031a0c08071000180030005019600020003000680070007800800100880100980100422e0a097265706f72745f696410041a0c0801104018003000501460002000300068007000780080010088010098010042330a0e61745f7269736b5f72616e67657310051a0c08011040180030005014600020003000680070007800800100880100980100480652a5010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f696422086c6f63616c6974792a097265706f72745f69642a0e61745f7269736b5f72616e6765733001300230034000400040004a10080010001a00200028003000380040005a00700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201510a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a086c6f63616c6974791a097265706f72745f69641a0e61745f7269736b5f72616e676573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a38a89","value":"030a83070a117265706c69636174696f6e5f7374617473181b200128013a00422c0a077a6f6e655f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a7375627a6f6e655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422e0a097265706f72745f696410031a0c0801104018003000501460002000300068007000780080010088010098010042310a0c746f74616c5f72616e67657310041a0c0801104018003000501460002000300068007000780080010088010098010042370a12756e617661696c61626c655f72616e67657310051a0c08011040180030005014600020003000680070007800800100880100980100423c0a17756e6465725f7265706c6963617465645f72616e67657310061a0c08011040180030005014600020003000680070007800800100880100980100423b0a166f7665725f7265706c6963617465645f72616e67657310071a0c08011040180030005014600020003000680070007800800100880100980100480852e0010a077072696d6172791001180122077a6f6e655f6964220a7375627a6f6e655f69642a097265706f72745f69642a0c746f74616c5f72616e6765732a12756e617661696c61626c655f72616e6765732a17756e6465725f7265706c6963617465645f72616e6765732a166f7665725f7265706c6963617465645f72616e67657330013002400040004a10080010001a00200028003000380040005a00700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2018e010a077072696d61727910001a077a6f6e655f69641a0a7375627a6f6e655f69641a097265706f72745f69641a0c746f74616c5f72616e6765731a12756e617661696c61626c655f72616e6765731a17756e6465725f7265706c6963617465645f72616e6765731a166f7665725f7265706c6963617465645f72616e67657320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a48a89","value":"030a82030a0c7265706f7274735f6d657461181c200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0967656e65726174656410021a0d080910001800300050a009600020003000680070007800800100880100980100480352700a077072696d61727910011801220269642a0967656e657261746564300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201200a077072696d61727910001a0269641a0967656e657261746564200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a68a89","value":"030aa1040a096e616d657370616365181e200128013a00422d0a08706172656e74494410011a0c0801104018003000501460002000300068007000780080010088010098010042330a0e706172656e74536368656d61494410021a0c0801104018003000501460002000300068007000780080010088010098010042290a046e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042270a02696410041a0c080110401800300050146000200130006800700078008001008801009801004805528d010a077072696d617279100118012208706172656e744944220e706172656e74536368656d61494422046e616d652a0269643001300230034000400040004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201330a077072696d61727910001a08706172656e7449441a0e706172656e74536368656d6149441a046e616d652001200220032800b201140a0866616d5f345f696410041a02696420042804b80105c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a78a89","value":"030aab050a1170726f7465637465645f74735f6d657461181f200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422c0a0776657273696f6e10021a0c0801104018003000501460002000300068007000780080010088010098010042300a0b6e756d5f7265636f72647310031a0c08011040180030005014600020003000680070007800800100880100980100422e0a096e756d5f7370616e7310041a0c0801104018003000501460002000300068007000780080010088010098010042300a0b746f74616c5f627974657310051a0c08011040180030005014600020003000680070007800800100880100980100480652a0010a077072696d61727910011801220973696e676c65746f6e2a0776657273696f6e2a0b6e756d5f7265636f7264732a096e756d5f7370616e732a0b746f74616c5f6279746573300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201260a0973696e676c65746f6e120f636865636b5f73696e676c65746f6e18002801300038004002b201500a077072696d61727910001a0973696e676c65746f6e1a0776657273696f6e1a0b6e756d5f7265636f7264731a096e756d5f7370616e731a0b746f74616c5f6279746573200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a88a89","value":"030a8f060a1470726f7465637465645f74735f7265636f7264731820200128013a0042280a02696410011a0d080e10001800300050861760002000300068007000780080010088010098010042280a02747310021a0d080310001800300050a40d600020003000680070007800800100880100980100422e0a096d6574615f7479706510031a0c0807100018003000501960002000300068007000780080010088010098010042290a046d65746110041a0c08081000180030005011600020013000680070007800800100880100980100422e0a096e756d5f7370616e7310051a0c08011040180030005014600020003000680070007800800100880100980100422a0a057370616e7310061a0c0808100018003000501160002000300068007000780080010088010098010042340a08766572696669656410071a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100422b0a0674617267657410081a0c08081000180030005011600020013000680070007800800100880100980100480952aa010a077072696d61727910011801220269642a0274732a096d6574615f747970652a046d6574612a096e756d5f7370616e732a057370616e732a0876657269666965642a06746172676574300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2015a0a077072696d61727910001a0269641a0274731a096d6574615f747970651a046d6574611a096e756d5f7370616e731a057370616e731a0876657269666965641a06746172676574200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89a98a89","value":"030a87050a0c726f6c655f6f7074696f6e731821200128013a00422d0a08757365726e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422b0a066f7074696f6e10021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c08071000180030005019600020013000680070007800800100880100980100422c0a07757365725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055289010a077072696d617279100118012208757365726e616d6522066f7074696f6e2a0576616c75652a07757365725f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a760a1175736572735f757365725f69645f696478100218002207757365725f696430043801380240004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201370a077072696d61727910001a08757365726e616d651a066f7074696f6e1a0576616c75651a07757365725f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89aa8a89","value":"030add030a1773746174656d656e745f62756e646c655f6368756e6b731822200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042300a0b6465736372697074696f6e10021a0c0807100018003000501960002001300068007000780080010088010098010042290a046461746110031a0c080810001800300050116000200030006800700078008001008801009801004804527a0a077072696d61727910011801220269642a0b6465736372697074696f6e2a0464617461300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012a0a077072696d61727910001a0269641a0b6465736372697074696f6e1a04646174612001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ab8a89","value":"030ae80c0a1e73746174656d656e745f646961676e6f73746963735f72657175657374731823200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042350a09636f6d706c6574656410021a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410031a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f646961676e6f73746963735f696410041a0c0801104018003000501460002001300068007000780080010088010098010042320a0c7265717565737465645f617410051a0d080910001800300050a00960002000300068007000780080010088010098010042410a156d696e5f657865637574696f6e5f6c6174656e637910061a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042300a0a657870697265735f617410071a0d080910001800300050a009600020013000680070007800800100880100980100423a0a1473616d706c696e675f70726f626162696c69747910081a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a09706c616e5f6769737410091a0c0807100018003000501960002001300068007000780080010088010098010042330a0e616e74695f706c616e5f67697374100a1a0c0800100018003000501060002001300068007000780080010088010098010042340a087265646163746564100b1a0c08001000180030005010600020002a0566616c73653000680070007800800100880100980100480c529f020a077072696d61727910011801220269642a09636f6d706c657465642a1573746174656d656e745f66696e6765727072696e742a1873746174656d656e745f646961676e6f73746963735f69642a0c7265717565737465645f61742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f676973742a087265646163746564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005afb010a0d636f6d706c657465645f696478100218002209636f6d706c65746564220269642a1573746174656d656e745f66696e6765727072696e742a156d696e5f657865637574696f6e5f6c6174656e63792a0a657870697265735f61742a1473616d706c696e675f70726f626162696c6974792a09706c616e5f676973742a0e616e74695f706c616e5f676973742a08726564616374656430023001400040004a10080010001a00200028003000380040005a0070037006700770087009700a700b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201620a3a73616d706c696e675f70726f626162696c697479204245545745454e20302e303a3a3a464c4f41543820414e4420312e303a3a3a464c4f415438121a636865636b5f73616d706c696e675f70726f626162696c69747918002808300038004002b201cf010a077072696d61727910001a0269641a09636f6d706c657465641a1573746174656d656e745f66696e6765727072696e741a1873746174656d656e745f646961676e6f73746963735f69641a0c7265717565737465645f61741a156d696e5f657865637574696f6e5f6c6174656e63791a0a657870697265735f61741a1473616d706c696e675f70726f626162696c6974791a09706c616e5f676973741a0e616e74695f706c616e5f676973741a087265646163746564200120022003200420052006200720082009200a200b2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ac8a89","value":"030ac1060a1573746174656d656e745f646961676e6f73746963731824200128013a0042370a02696410011a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100423a0a1573746174656d656e745f66696e6765727072696e7410021a0c08071000180030005019600020003000680070007800800100880100980100422e0a0973746174656d656e7410031a0c0807100018003000501960002000300068007000780080010088010098010042320a0c636f6c6c65637465645f617410041a0d080910001800300050a009600020003000680070007800800100880100980100422b0a05747261636510051a0d081210001800300050da1d60002001300068007000780080010088010098010042430a0d62756e646c655f6368756e6b7310061a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a056572726f7210071a0c08071000180030005019600020013000680070007800800100880100980100480852bc010a077072696d61727910011801220269642a1573746174656d656e745f66696e6765727072696e742a0973746174656d656e742a0c636f6c6c65637465645f61742a0574726163652a0d62756e646c655f6368756e6b732a056572726f72300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2016c0a077072696d61727910001a0269641a1573746174656d656e745f66696e6765727072696e741a0973746174656d656e741a0c636f6c6c65637465645f61741a0574726163651a0d62756e646c655f6368756e6b731a056572726f7220012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ad8a89","value":"030ad6090a0e7363686564756c65645f6a6f62731825200128013a0042400a0b7363686564756c655f696410011a0c08011040180030005014600020002a0e756e697175655f726f7769642829300068007000780080010088010098010042320a0d7363686564756c655f6e616d6510021a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210041a0c08071000180030005019600020003000680070007800800100880100980100422e0a086e6578745f72756e10051a0d080910001800300050a00960002001300068007000780080010088010098010042330a0e7363686564756c655f737461746510061a0c0808100018003000501160002001300068007000780080010088010098010042320a0d7363686564756c655f6578707210071a0c0807100018003000501960002001300068007000780080010088010098010042350a107363686564756c655f64657461696c7310081a0c0808100018003000501160002001300068007000780080010088010098010042320a0d6578656375746f725f7479706510091a0c0807100018003000501960002000300068007000780080010088010098010042330a0e657865637574696f6e5f61726773100a1a0c08081000180030005011600020003000680070007800800100880100980100480b52f7010a077072696d61727910011801220b7363686564756c655f69642a0d7363686564756c655f6e616d652a07637265617465642a056f776e65722a086e6578745f72756e2a0e7363686564756c655f73746174652a0d7363686564756c655f657870722a107363686564756c655f64657461696c732a0d6578656375746f725f747970652a0e657865637574696f6e5f61726773300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a700a0c6e6578745f72756e5f6964781002180022086e6578745f72756e3005380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201380a05736368656410001a0b7363686564756c655f69641a086e6578745f72756e1a0e7363686564756c655f73746174652001200520062800b201780a056f7468657210011a0d7363686564756c655f6e616d651a07637265617465641a056f776e65721a0d7363686564756c655f657870721a107363686564756c655f64657461696c731a0d6578656375746f725f747970651a0e657865637574696f6e5f61726773200220032004200720082009200a2800b80102c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89af8a89","value":"030aef030a0b73716c6c6976656e6573731827200128013a00422f0a0a73657373696f6e5f696410011a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10021a0d080310001800300050a40d60002000300068007000780080010088010098010042300a0b637264625f726567696f6e10031a0c080810001800300050116000200030006800700078008001008801009801004804528a010a077072696d61727910021801220b637264625f726567696f6e220a73657373696f6e5f69642a0a65787069726174696f6e30033001400040004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201380a077072696d61727910001a0b637264625f726567696f6e1a0a73657373696f6e5f69641a0a65787069726174696f6e2003200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b08a89","value":"030adc040a0a6d6967726174696f6e731828200128013a00422a0a056d616a6f7210011a0c08011040180030005014600020003000680070007800800100880100980100422a0a056d696e6f7210021a0c08011040180030005014600020003000680070007800800100880100980100422a0a05706174636810031a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e7465726e616c10041a0c0801104018003000501460002000300068007000780080010088010098010042320a0c636f6d706c657465645f617410051a0d080910001800300050a0096000200030006800700078008001008801009801004806529a010a077072696d6172791001180122056d616a6f7222056d696e6f72220570617463682208696e7465726e616c2a0c636f6d706c657465645f6174300130023003300440004000400040004a10080010001a00200028003000380040005a0070057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201440a077072696d61727910001a056d616a6f721a056d696e6f721a0570617463681a08696e7465726e616c1a0c636f6d706c657465645f6174200120022003200420052805b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b18a89","value":"030ac6030a0b6a6f696e5f746f6b656e731829200128013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422b0a0673656372657410021a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10031a0d080910001800300050a0096000200030006800700078008001008801009801004804527b0a077072696d61727910011801220269642a067365637265742a0a65787069726174696f6e300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012b0a077072696d61727910001a0269641a067365637265741a0a65787069726174696f6e2001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b28a89","value":"030a92280a1473746174656d656e745f73746174697374696373182a200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410061a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10071a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110081a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310091a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e100a1a0d081210001800300050da1d600020003000680070007800800100880100980100429c020a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38100b1a0c080110201800300050176000200030015a8c016d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f69642c20706c616e5f686173682c207472616e73616374696f6e5f66696e6765727072696e745f696429292c20383a3a3a494e543829680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100c1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d3000680070007800800100880100980100426c0a0d696e64657865735f7573616765100d1a0d081210001800300050da1d6000200130005a3728737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27696e6465786573273a3a3a535452494e4768007000780080010188010098010042710a0f657865637574696f6e5f636f756e74100e1a0c080110401800300050146000200130005a3b2828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27636e74273a3a3a535452494e47293a3a494e5438680070007800800100880100980100428a010a0f736572766963655f6c6174656e6379100f1a0d080210401800300050bd056000200130005a53282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f4154386800700078008001008801009801004297010a0d6370755f73716c5f6e616e6f7310101a0d080210401800300050bd056000200130005a62282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e2763707553514c4e616e6f73273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100429c010a0f636f6e74656e74696f6e5f74696d6510111a0d080210401800300050bd056000200130005a65282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e27636f6e74656e74696f6e54696d65273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f41543868007000780080010088010098010042dc010a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d6510121a0d080210401800300050bd056000200130005a95012828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e3e27636e74273a3a3a535452494e47293a3a464c4f415438202a20282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100428a010a0b7039395f6c6174656e637910131a0d080210401800300050bd056000200130005a57282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e276c6174656e6379496e666f273a3a3a535452494e47292d3e27703939273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100481452c2050a077072696d617279100118012268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e732a0f657865637574696f6e5f636f756e742a0f736572766963655f6c6174656e63792a0d6370755f73716c5f6e616e6f732a0f636f6e74656e74696f6e5f74696d652a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d652a0b7039395f6c6174656e6379300b30013002300330043005300640004000400040004000400040004a10080010001a00200028003000380040005a00700770087009700a700c700e700f70107011701270137a0408002000800100880100900104980101a201c70108011268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f69642209706c616e5f68617368221a7472616e73616374696f6e5f66696e6765727072696e745f6964a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa7010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380b3801380438053806400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a89010a11696e64657865735f75736167655f69647810031800220d696e64657865735f7573616765300d380b38013802380338043805380640004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e00100e90100000000000000005acf010a13657865637574696f6e5f636f756e745f69647810041800220d616767726567617465645f747322086170705f6e616d65220f657865637574696f6e5f636f756e7430013005300e380b38023803380438064000400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ad1010a13736572766963655f6c6174656e63795f69647810051800220d616767726567617465645f747322086170705f6e616d65220f736572766963655f6c6174656e637930013005300f380b38023803380438064000400040014a10080010001a00200028003000380040005a00680f7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a116370755f73716c5f6e616e6f735f69647810061800220d616767726567617465645f747322086170705f6e616d65220d6370755f73716c5f6e616e6f73300130053010380b38023803380438064000400040014a10080010001a00200028003000380040005a0068107a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ad1010a13636f6e74656e74696f6e5f74696d655f69647810071800220d616767726567617465645f747322086170705f6e616d65220f636f6e74656e74696f6e5f74696d65300130053011380b38023803380438064000400040014a10080010001a00200028003000380040005a0068117a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005aef010a22746f74616c5f657374696d617465645f657865637574696f6e5f74696d655f69647810081800220d616767726567617465645f747322086170705f6e616d65221e746f74616c5f657374696d617465645f657865637574696f6e5f74696d65300130053012380b38023803380438064000400040014a10080010001a00200028003000380040005a0068127a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac9010a0f7039395f6c6174656e63795f69647810091800220d616767726567617465645f747322086170705f6e616d65220b7039395f6c6174656e6379300130053013380b38023803380438064000400040014a10080010001a00200028003000380040005a0068137a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e9010000000000000000600a6a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201b9020abc01637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e543829126e636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381800280b300038014002b201a4030a077072696d61727910001a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e731a0f657865637574696f6e5f636f756e741a0f736572766963655f6c6174656e63791a0d6370755f73716c5f6e616e6f731a0f636f6e74656e74696f6e5f74696d651a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d651a0b7039395f6c6174656e6379200b200120022003200420052006200720082009200a200c200e200f20102011201220132800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b38a89","value":"030a98200a167472616e73616374696f6e5f73746174697374696373182b200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410041a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10051a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110061a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310071a0d081210001800300050da1d60002000300068007000780080010088010098010042cf010a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3810081a0c080110201800300050176000200030015a656d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f696429292c20383a3a3a494e54382968007000780080010088010098010042710a0f657865637574696f6e5f636f756e7410091a0c080110401800300050146000200130005a3b2828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27636e74273a3a3a535452494e47293a3a494e5438680070007800800100880100980100428a010a0f736572766963655f6c6174656e6379100a1a0d080210401800300050bd056000200130005a53282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f4154386800700078008001008801009801004297010a0d6370755f73716c5f6e616e6f73100b1a0d080210401800300050bd056000200130005a62282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e2763707553514c4e616e6f73273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100429c010a0f636f6e74656e74696f6e5f74696d65100c1a0d080210401800300050bd056000200130005a65282828737461746973746963732d3e27657865637574696f6e5f73746174697374696373273a3a3a535452494e47292d3e27636f6e74656e74696f6e54696d65273a3a3a535452494e47292d3e276d65616e273a3a3a535452494e47293a3a464c4f41543868007000780080010088010098010042dc010a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d65100d1a0d080210401800300050bd056000200130005a95012828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e3e27636e74273a3a3a535452494e47293a3a464c4f415438202a20282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e277376634c6174273a3a3a535452494e47292d3e3e276d65616e273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100428a010a0b7039395f6c6174656e6379100e1a0d080210401800300050bd056000200130005a57282828737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e276c6174656e6379496e666f273a3a3a535452494e47292d3e27703939273a3a3a535452494e47293a3a464c4f415438680070007800800100880100980100480f5280040a077072696d617279100118012243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a0f657865637574696f6e5f636f756e742a0f736572766963655f6c6174656e63792a0d6370755f73716c5f6e616e6f732a0f636f6e74656e74696f6e5f74696d652a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d652a0b7039395f6c6174656e637930083001300230033004400040004000400040004a10080010001a00200028003000380040005a007005700670077009700a700b700c700d700e7a0408002000800100880100900104980101a2017b08011243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f6964a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a85010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f69643002380838013803380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005acb010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f747322086170705f6e616d65220f657865637574696f6e5f636f756e743001300330093808380238044000400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a13736572766963655f6c6174656e63795f69647810041800220d616767726567617465645f747322086170705f6e616d65220f736572766963655f6c6174656e637930013003300a3808380238044000400040014a10080010001a00200028003000380040005a00680a7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac9010a116370755f73716c5f6e616e6f735f69647810051800220d616767726567617465645f747322086170705f6e616d65220d6370755f73716c5f6e616e6f7330013003300b3808380238044000400040014a10080010001a00200028003000380040005a00680b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005acd010a13636f6e74656e74696f6e5f74696d655f69647810061800220d616767726567617465645f747322086170705f6e616d65220f636f6e74656e74696f6e5f74696d6530013003300c3808380238044000400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005aeb010a22746f74616c5f657374696d617465645f657865637574696f6e5f74696d655f69647810071800220d616767726567617465645f747322086170705f6e616d65221e746f74616c5f657374696d617465645f657865637574696f6e5f74696d6530013003300d3808380238044000400040014a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e90100000000000000005ac5010a0f7039395f6c6174656e63795f69647810081800220d616767726567617465645f747322086170705f6e616d65220b7039395f6c6174656e637930013003300e3808380238044000400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba01286170705f6e616d65204e4f54204c494b4520272420696e7465726e616c25273a3a3a535452494e47c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100a201ef010a9701637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e5438291249636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3818002808300038014002b201b3020a077072696d61727910001a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a0f657865637574696f6e5f636f756e741a0f736572766963655f6c6174656e63791a0d6370755f73716c5f6e616e6f731a0f636f6e74656e74696f6e5f74696d651a1e746f74616c5f657374696d617465645f657865637574696f6e5f74696d651a0b7039395f6c6174656e6379200820012002200320042005200620072009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b48a89","value":"030af6050a1664617461626173655f726f6c655f73657474696e6773182c200128013a0042300a0b64617461626173655f696410011a0c080c100018003000501a600020003000680070007800800100880100980100422e0a09726f6c655f6e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100423e0a0873657474696e677310031a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a60002000300068007000780080010088010098010048055292010a077072696d61727910011801220b64617461626173655f69642209726f6c655f6e616d652a0873657474696e67732a07726f6c655f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005aae010a2e64617461626173655f726f6c655f73657474696e67735f64617461626173655f69645f726f6c655f69645f6b657910021801220b64617461626173655f69642207726f6c655f69642a0873657474696e6773300130043802400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201400a077072696d61727910001a0b64617461626173655f69641a09726f6c655f6e616d651a0873657474696e67731a07726f6c655f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b58a89","value":"030ac30b0a0c74656e616e745f7573616765182d200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042300a0b696e7374616e63655f696410021a0c0801104018003000501460002000300068007000780080010088010098010042350a106e6578745f696e7374616e63655f696410031a0c0801104018003000501460002000300068007000780080010088010098010042310a0b6c6173745f75706461746510041a0d080510001800300050da0860002000300068007000780080010088010098010042340a0e72755f62757273745f6c696d697410051a0d080210401800300050bd0560002001300068007000780080010088010098010042340a0e72755f726566696c6c5f7261746510061a0d080210401800300050bd0560002001300068007000780080010088010098010042300a0a72755f63757272656e7410071a0d080210401800300050bd0560002001300068007000780080010088010098010042370a1163757272656e745f73686172655f73756d10081a0d080210401800300050bd0560002001300068007000780080010088010098010042360a11746f74616c5f636f6e73756d7074696f6e10091a0c0808100018003000501160002001300068007000780080010088010098010042330a0e696e7374616e63655f6c65617365100a1a0c0808100018003000501160002001300068007000780080010088010098010042310a0c696e7374616e63655f736571100b1a0c0801104018003000501460002001300068007000780080010088010098010042350a0f696e7374616e63655f736861726573100c1a0d080210401800300050bd0560002001300068007000780080010088010098010042320a0d63757272656e745f7261746573100d1a0c08081000180030005011600020013000680070007800800100880100980100422f0a0a6e6578745f7261746573100e1a0c08081000180030005011600020013000680070007800800100880100980100480f52ce020a077072696d61727910011801220974656e616e745f6964220b696e7374616e63655f69642a106e6578745f696e7374616e63655f69642a0b6c6173745f7570646174652a0e72755f62757273745f6c696d69742a0e72755f726566696c6c5f726174652a0a72755f63757272656e742a1163757272656e745f73686172655f73756d2a11746f74616c5f636f6e73756d7074696f6e2a0e696e7374616e63655f6c656173652a0c696e7374616e63655f7365712a0f696e7374616e63655f7368617265732a0d63757272656e745f72617465732a0a6e6578745f726174657330013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c700d700e7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201fc010a077072696d61727910001a0974656e616e745f69641a0b696e7374616e63655f69641a106e6578745f696e7374616e63655f69641a0b6c6173745f7570646174651a0e72755f62757273745f6c696d69741a0e72755f726566696c6c5f726174651a0a72755f63757272656e741a1163757272656e745f73686172655f73756d1a11746f74616c5f636f6e73756d7074696f6e1a0e696e7374616e63655f6c656173651a0c696e7374616e63655f7365711a0f696e7374616e63655f7368617265731a0d63757272656e745f72617465731a0a6e6578745f7261746573200120022003200420052006200720082009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b68a89","value":"030acb060a0d73716c5f696e7374616e636573182e200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046164647210021a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410031a0c08081000180030005011600020013000680070007800800100880100980100422e0a086c6f63616c69747910041a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0873716c5f6164647210051a0c0807100018003000501960002001300068007000780080010088010098010042300a0b637264625f726567696f6e10061a0c0808100018003000501160002000300068007000780080010088010098010042330a0e62696e6172795f76657273696f6e10071a0c0807100018003000501960002001300068007000780080010088010098010042300a0b69735f647261696e696e6710081a0c08001000180030005010600020013000680070007800800100880100980100480952c3010a077072696d61727910021801220b637264625f726567696f6e220269642a04616464722a0a73657373696f6e5f69642a086c6f63616c6974792a0873716c5f616464722a0e62696e6172795f76657273696f6e2a0b69735f647261696e696e6730063001400040004a10080010001a00200028003000380040005a007002700370047005700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201710a077072696d61727910001a0269641a04616464721a0a73657373696f6e5f69641a086c6f63616c6974791a0873716c5f616464721a0b637264625f726567696f6e1a0e62696e6172795f76657273696f6e1a0b69735f647261696e696e67200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b78a89","value":"030a8a040a137370616e5f636f6e66696775726174696f6e73182f200128013a00422e0a0973746172745f6b657910011a0c08081000180030005011600020003000680070007800800100880100980100422c0a07656e645f6b657910021a0c08081000180030005011600020003000680070007800800100880100980100422b0a06636f6e66696710031a0c080810001800300050116000200030006800700078008001008801009801004804527f0a077072696d61727910011801220973746172745f6b65792a07656e645f6b65792a06636f6e666967300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a2012f0a1373746172745f6b6579203c20656e645f6b6579120c636865636b5f626f756e6473180028012802300038004002b2012f0a077072696d61727910001a0973746172745f6b65791a07656e645f6b65791a06636f6e6669672001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800301880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89b88a89","value":"030adc020a0b726f6c655f69645f7365711830200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a250a0d0a0561646d696e10a00618a0060a0c0a04726f6f7410a00618a00612046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011a0801106418ffffffff0720642800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ba8a89","value":"030af3050a0f74656e616e745f73657474696e67731832200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c0807100018003000501960002000300068007000780080010088010098010042450a0c6c6173745f7570646174656410041a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422f0a0a76616c75655f7479706510051a0c08071000180030005019600020003000680070007800800100880100980100422b0a06726561736f6e10061a0c08071000180030005019600020013000680070007800800100880100980100480752a5010a077072696d61727910011801220974656e616e745f696422046e616d652a0576616c75652a0c6c6173745f757064617465642a0a76616c75655f747970652a06726561736f6e30013002400040004a10080010001a00200028003000380040005a0070037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b20185010a3966616d5f305f74656e616e745f69645f6e616d655f76616c75655f6c6173745f757064617465645f76616c75655f747970655f726561736f6e10001a0974656e616e745f69641a046e616d651a0576616c75651a0c6c6173745f757064617465641a0a76616c75655f747970651a06726561736f6e2001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89bb8a89","value":"030ac1030a0a7370616e5f636f756e741833200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422f0a0a7370616e5f636f756e7410021a0c08011040180030005014600020003000680070007800800100880100980100480352780a077072696d61727910011801220973696e676c65746f6e2a0a7370616e5f636f756e74300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201210a0973696e676c65746f6e120a73696e676c655f726f7718002801300038004002b201280a077072696d61727910001a0973696e676c65746f6e1a0a7370616e5f636f756e74200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89bc8a89","value":"030ae2070a0a70726976696c656765731834200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042290a047061746810021a0c0807100018003000501960002000300068007000780080010088010098010042400a0a70726976696c6567657310031a1d080f100018003000380750f1075a0c08071000180030005019600060002000300068007000780080010088010098010042430a0d6772616e745f6f7074696f6e7310041a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07757365725f696410051a0c080c100018003000501a6000200030006800700078008001008801009801004806529d010a077072696d617279100118012208757365726e616d652204706174682a0a70726976696c656765732a0d6772616e745f6f7074696f6e732a07757365725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00103e00100e90100000000000000005aa7010a1b70726976696c656765735f706174685f757365725f69645f6b6579100218012204706174682207757365725f69642a0a70726976696c656765732a0d6772616e745f6f7074696f6e73300230053801400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa7010a1c70726976696c656765735f706174685f757365726e616d655f6b6579100318012204706174682208757365726e616d652a0a70726976696c656765732a0d6772616e745f6f7074696f6e7330023001400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2014b0a077072696d61727910001a08757365726e616d651a04706174681a0a70726976696c656765731a0d6772616e745f6f7074696f6e731a07757365725f6964200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880304a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89bd8a89","value":"030add060a1465787465726e616c5f636f6e6e656374696f6e731835200128013a0042340a0f636f6e6e656374696f6e5f6e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042400a077570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f636f6e6e656374696f6e5f7479706510041a0c0807100018003000501960002000300068007000780080010088010098010042370a12636f6e6e656374696f6e5f64657461696c7310051a0c08081000180030005011600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852c4010a077072696d61727910011801220f636f6e6e656374696f6e5f6e616d652a07637265617465642a07757064617465642a0f636f6e6e656374696f6e5f747970652a12636f6e6e656374696f6e5f64657461696c732a056f776e65722a086f776e65725f6964300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201740a077072696d61727910001a0f636f6e6e656374696f6e5f6e616d651a07637265617465641a07757064617465641a0f636f6e6e656374696f6e5f747970651a12636f6e6e656374696f6e5f64657461696c731a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89be8a89","value":"030aa3040a086a6f625f696e666f1836200128013a00422b0a066a6f625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e666f5f6b657910021a0c0807100018003000501960002000300068007000780080010088010098010042420a077772697474656e10031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a0576616c756510041a0c080810001800300050116000200130006800700078008001008801009801004805528b010a077072696d6172791001180122066a6f625f69642208696e666f5f6b657922077772697474656e2a0576616c75653001300230034000400040014a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201370a077072696d61727910001a066a6f625f69641a08696e666f5f6b65791a077772697474656e1a0576616c756520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89bf8a89","value":"030a9f040a167370616e5f73746174735f756e697175655f6b6579731837200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422e0a096b65795f627974657310021a0c08081000180030005011600020013000680070007800800100880100980100480352700a077072696d61727910011801220269642a096b65795f6279746573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a7e0a19756e697175655f6b6579735f6b65795f62797465735f6964781002180122096b65795f62797465733002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201200a077072696d61727910001a0269641a096b65795f6279746573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c08a89","value":"030a82060a127370616e5f73746174735f6275636b6574731838200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422f0a0973616d706c655f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042320a0c73746172745f6b65795f696410031a0d080e10001800300050861760002000300068007000780080010088010098010042300a0a656e645f6b65795f696410041a0d080e100018003000508617600020003000680070007800800100880100980100422d0a08726571756573747310051a0c080110401800300050146000200030006800700078008001008801009801004806529a010a077072696d61727910011801220269642a0973616d706c655f69642a0c73746172745f6b65795f69642a0a656e645f6b65795f69642a087265717565737473300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7a0a156275636b6574735f73616d706c655f69645f69647810021800220973616d706c655f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2014a0a077072696d61727910001a0269641a0973616d706c655f69641a0c73746172745f6b65795f69641a0a656e645f6b65795f69641a087265717565737473200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c18a89","value":"030ab5040a127370616e5f73746174735f73616d706c65731839200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f757569642829300068007000780080010088010098010042440a0b73616d706c655f74696d6510021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100480352720a077072696d61727910011801220269642a0b73616d706c655f74696d65300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e00100e90100000000000000005a7e0a1773616d706c65735f73616d706c655f74696d655f69647810021801220b73616d706c655f74696d653002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201220a077072696d61727910001a0269641a0b73616d706c655f74696d65200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c28a89","value":"030aa9030a1c7370616e5f73746174735f74656e616e745f626f756e646172696573183a200128013a00422e0a0974656e616e745f696410011a0c08011040180030005014600020003000680070007800800100880100980100422f0a0a626f756e64617269657310021a0c08081000180030005011600020003000680070007800800100880100980100480352780a077072696d61727910011801220974656e616e745f69642a0a626f756e646172696573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201280a077072696d61727910001a0974656e616e745f69641a0a626f756e646172696573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c38a89","value":"030ab1060a0d7461736b5f7061796c6f616473183b200128013a0042270a02696410011a0c0807100018003000501960002000300068007000780080010088010098010042420a076372656174656410021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210031a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410041a0c080c100018003000501a60002000300068007000780080010088010098010042300a0b6d696e5f76657273696f6e10051a0c0807100018003000501960002000300068007000780080010088010098010042300a0b6465736372697074696f6e10061a0c0807100018003000501960002001300068007000780080010088010098010042290a047479706510071a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510081a0c08081000180030005011600020003000680070007800800100880100980100480952b2010a077072696d61727910011801220269642a07637265617465642a056f776e65722a086f776e65725f69642a0b6d696e5f76657273696f6e2a0b6465736372697074696f6e2a04747970652a0576616c7565300140004a10080010001a00200028003000380040005a0070027003700470057006700770087a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201620a077072696d61727910001a0269641a07637265617465641a056f776e65721a086f776e65725f69641a0b6d696e5f76657273696f6e1a0b6465736372697074696f6e1a04747970651a0576616c7565200120022003200420052006200720082800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c48a89","value":"030a82060a0c74656e616e745f7461736b73183c200128013a00422e0a0974656e616e745f696410011a0c08011040180030005014600020003000680070007800800100880100980100422b0a0669737375657210021a0c08071000180030005019600020003000680070007800800100880100980100422c0a077461736b5f696410031a0c0801104018003000501460002000300068007000780080010088010098010042420a076372656174656410041a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422f0a0a7061796c6f61645f696410051a0c08071000180030005019600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852b1010a077072696d61727910011801220974656e616e745f6964220669737375657222077461736b5f69642a07637265617465642a0a7061796c6f61645f69642a056f776e65722a086f776e65725f69643001300230034000400040004a10080010001a00200028003000380040005a0070047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2015d0a077072696d61727910001a0974656e616e745f69641a066973737565721a077461736b5f69641a07637265617465641a0a7061796c6f61645f69641a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c58a89","value":"030af4180a1273746174656d656e745f6163746976697479183d200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c0807100018003000501960002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10061a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110071a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310081a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e10091a0d081210001800300050da1d600020003000680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100a1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d300068007000780080010088010098010042340a0f657865637574696f6e5f636f756e74100b1a0c08011040180030005014600020003000680070007800800100880100980100423d0a17657865637574696f6e5f746f74616c5f7365636f6e6473100c1a0d080210401800300050bd0560002000300068007000780080010088010098010042450a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e6473100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e6473100e1a0d080210401800300050bd0560002000300068007000780080010088010098010042370a116370755f73716c5f6176675f6e616e6f73100f1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f6176675f7365636f6e647310101a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f7039395f7365636f6e647310111a0d080210401800300050bd05600020003000680070007800800100880100980100481252cd030a077072696d61727910011801220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d652a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e732a0f657865637574696f6e5f636f756e742a17657865637574696f6e5f746f74616c5f7365636f6e64732a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64732a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64732a116370755f73716c5f6176675f6e616e6f732a1b736572766963655f6c6174656e63795f6176675f7365636f6e64732a1b736572766963655f6c6174656e63795f7039395f7365636f6e647330013002300330043005400040004000400040004a10080010001a00200028003000380040005a007006700770087009700a700b700c700d700e700f701070117a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa0010a1266696e6765727072696e745f69645f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380138043805400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a97010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f7473220f657865637574696f6e5f636f756e743001300b3802380338043805400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aa9010a1b657865637574696f6e5f746f74616c5f7365636f6e64735f69647810041800220d616767726567617465645f74732217657865637574696f6e5f746f74616c5f7365636f6e64733001300c3802380338043805400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64735f69647810051800220d616767726567617465645f7473221b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64733001300e3802380338043805400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a9d010a156370755f73716c5f6176675f6e616e6f735f69647810061800220d616767726567617465645f747322116370755f73716c5f6176675f6e616e6f733001300f3802380338043805400040014a10080010001a00200028003000380040005a00680f7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f736572766963655f6c6174656e63795f6176675f7365636f6e64735f69647810071800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f6176675f7365636f6e6473300130103802380338043805400040014a10080010001a00200028003000380040005a0068107a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab1010a1f736572766963655f6c6174656e63795f7039395f7365636f6e64735f69647810081800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f7039395f7365636f6e6473300130113802380338043805400040014a10080010001a00200028003000380040005a0068117a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201f5020a077072696d61727910001a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e731a0f657865637574696f6e5f636f756e741a17657865637574696f6e5f746f74616c5f7365636f6e64731a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64731a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64731a116370755f73716c5f6176675f6e616e6f731a1b736572766963655f6c6174656e63795f6176675f7365636f6e64731a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473200120022003200420052006200720082009200a200b200c200d200e200f201020112800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c68a89","value":"030adf150a147472616e73616374696f6e5f6163746976697479183e200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10041a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110051a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310061a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a05717565727910071a0c0807100018003000501960002000300068007000780080010088010098010042340a0f657865637574696f6e5f636f756e7410081a0c08011040180030005014600020003000680070007800800100880100980100423d0a17657865637574696f6e5f746f74616c5f7365636f6e647310091a0d080210401800300050bd0560002000300068007000780080010088010098010042450a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e6473100a1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e6473100b1a0d080210401800300050bd0560002000300068007000780080010088010098010042370a116370755f73716c5f6176675f6e616e6f73100c1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f6176675f7365636f6e6473100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042410a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473100e1a0d080210401800300050bd05600020003000680070007800800100880100980100480f5286030a077072696d61727910011801220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d652a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a0571756572792a0f657865637574696f6e5f636f756e742a17657865637574696f6e5f746f74616c5f7365636f6e64732a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64732a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64732a116370755f73716c5f6176675f6e616e6f732a1b736572766963655f6c6174656e63795f6176675f7365636f6e64732a1b736572766963655f6c6174656e63795f7039395f7365636f6e64733001300230034000400040004a10080010001a00200028003000380040005a00700470057006700770087009700a700b700c700d700e7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7e0a1266696e6765727072696e745f69645f69647810021800220e66696e6765727072696e745f696430023801380340004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a93010a13657865637574696f6e5f636f756e745f69647810031800220d616767726567617465645f7473220f657865637574696f6e5f636f756e743001300838023803400040014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aa5010a1b657865637574696f6e5f746f74616c5f7365636f6e64735f69647810041800220d616767726567617465645f74732217657865637574696f6e5f746f74616c5f7365636f6e64733001300938023803400040014a10080010001a00200028003000380040005a0068097a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64735f69647810051800220d616767726567617465645f7473221b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64733001300b38023803400040014a10080010001a00200028003000380040005a00680b7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a99010a156370755f73716c5f6176675f6e616e6f735f69647810061800220d616767726567617465645f747322116370755f73716c5f6176675f6e616e6f733001300c38023803400040014a10080010001a00200028003000380040005a00680c7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f736572766963655f6c6174656e63795f6176675f7365636f6e64735f69647810071800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f6176675f7365636f6e64733001300d38023803400040014a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005aad010a1f736572766963655f6c6174656e63795f7039395f7365636f6e64735f69647810081800220d616767726567617465645f7473221b736572766963655f6c6174656e63795f7039395f7365636f6e64733001300e38023803400040014a10080010001a00200028003000380040005a00680e7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060096a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b201b2020a077072696d61727910001a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a0571756572791a0f657865637574696f6e5f636f756e741a17657865637574696f6e5f746f74616c5f7365636f6e64731a1f657865637574696f6e5f746f74616c5f636c75737465725f7365636f6e64731a1b636f6e74656e74696f6e5f74696d655f6176675f7365636f6e64731a116370755f73716c5f6176675f6e616e6f731a1b736572766963655f6c6174656e63795f6176675f7365636f6e64731a1b736572766963655f6c6174656e63795f7039395f7365636f6e6473200120022003200420052006200720082009200a200b200c200d200e2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c78a89","value":"030ade020a0d74656e616e745f69645f736571183f200128013a00422a0a0576616c756510011a0c08011040180030005014600020003000680070007800800100880100980100480052660a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060006a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011e0801100118ffffffffffffffff7f20012800320408001000380142004800e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c88a89","value":"030ac80a0a0f6d7663635f737461746973746963731840200128013a0042450a0a637265617465645f617410011a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042300a0b64617461626173655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422d0a087461626c655f696410031a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e6465785f696410041a0c0801104018003000501460002000300068007000780080010088010098010042300a0a7374617469737469637310051a0d081210001800300050da1d60002000300068007000780080010088010098010042ab010a3f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313610061a0c080110201800300050176000200030015a456d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328637265617465645f61742929292c2031363a3a3a494e543829680070007800800101880100980100480752e4020a146d7663635f737461746973746963735f706b657910011801223f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f3136220a637265617465645f6174220b64617461626173655f696422087461626c655f69642208696e6465785f69642a0a7374617469737469637330063001300230033004400040004000400040004a10080010001a00200028003000380040005a0070057a0408002000800100880100900104980101a201720801123f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f31361810220a637265617465645f6174220b64617461626173655f69642208696e6465785f696422087461626c655f6964a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a201bd020ae901637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291245636865636b5f637264625f696e7465726e616c5f637265617465645f61745f64617461626173655f69645f696e6465785f69645f7461626c655f69645f73686172645f313618002806300038014002b201500a077072696d61727910001a0a637265617465645f61741a0b64617461626173655f69641a087461626c655f69641a08696e6465785f69641a0a73746174697374696373200120022003200420052805b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89c98a89","value":"030abe170a1e7472616e73616374696f6e5f657865637574696f6e5f696e7369676874731841200128013a0042340a0e7472616e73616374696f6e5f696410011a0d080e100018003000508617600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410021a0c0808100018003000501160002000300068007000780080010088010098010042320a0d71756572795f73756d6d61727910031a0c0807100018003000501960002001300068007000780080010088010098010042310a0c696d706c696369745f74786e10041a0c08001000180030005010600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410051a0c0807100018003000501960002000300068007000780080010088010098010042300a0a73746172745f74696d6510061a0d080910001800300050a009600020013000680070007800800100880100980100422e0a08656e645f74696d6510071a0d080910001800300050a009600020013000680070007800800100880100980100422e0a09757365725f6e616d6510081a0c08071000180030005019600020013000680070007800800100880100980100422d0a086170705f6e616d6510091a0c0807100018003000501960002001300068007000780080010088010098010042320a0d757365725f7072696f72697479100a1a0c08071000180030005019600020013000680070007800800100880100980100422c0a0772657472696573100b1a0c0801104018003000501460002001300068007000780080010088010098010042360a116c6173745f72657472795f726561736f6e100c1a0c08071000180030005019600020013000680070007800800100880100980100423e0a0870726f626c656d73100d1a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100423c0a06636175736573100e1a1d080f104018003000380150f8075a0c08011040180030005014600060002001300068007000780080010088010098010042480a1273746d745f657865637574696f6e5f696473100f1a1d080f100018003000380750f1075a0c08071000180030005019600060002001300068007000780080010088010098010042320a0d6370755f73716c5f6e616e6f7310101a0c0801104018003000501460002001300068007000780080010088010098010042340a0f6c6173745f6572726f725f636f646510111a0c08071000180030005019600020013000680070007800800100880100980100422b0a0673746174757310121a0c08011040180030005014600020013000680070007800800100880100980100423b0a0f636f6e74656e74696f6e5f74696d6510131a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042350a0f636f6e74656e74696f6e5f696e666f10141a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0764657461696c7310151a0d081210001800300050da1d60002001300068007000780080010088010098010042420a076372656174656410161a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042a0010a2a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313610171a0c080110201800300050176000200030015a4f6d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328656e645f74696d652c2073746172745f74696d652929292c2031363a3a3a494e543829680070007800800101880100980100481852b6030a077072696d61727910011801220e7472616e73616374696f6e5f69642a1a7472616e73616374696f6e5f66696e6765727072696e745f69642a0d71756572795f73756d6d6172792a0c696d706c696369745f74786e2a0a73657373696f6e5f69642a0a73746172745f74696d652a08656e645f74696d652a09757365725f6e616d652a086170705f6e616d652a0d757365725f7072696f726974792a07726574726965732a116c6173745f72657472795f726561736f6e2a0870726f626c656d732a066361757365732a1273746d745f657865637574696f6e5f6964732a0d6370755f73716c5f6e616e6f732a0f6c6173745f6572726f725f636f64652a067374617475732a0f636f6e74656e74696f6e5f74696d652a0f636f6e74656e74696f6e5f696e666f2a0764657461696c732a0763726561746564300140004a10080010001a00200028003000380040005a0070027003700470057006700770087009700a700b700c700d700e700f70107011701270137014701570167a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a94010a1e7472616e73616374696f6e5f66696e6765727072696e745f69645f69647810021800221a7472616e73616374696f6e5f66696e6765727072696e745f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005af2010a0e74696d655f72616e67655f69647810031800222a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136220a73746172745f74696d652208656e645f74696d6530173006300738014000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a201460801122a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618102208656e645f74696d65220a73746172745f74696d65a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20193020ad401637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291230636865636b5f637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618002817300038014002b201e6020a077072696d61727910001a0e7472616e73616374696f6e5f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a0d71756572795f73756d6d6172791a0c696d706c696369745f74786e1a0a73657373696f6e5f69641a0a73746172745f74696d651a08656e645f74696d651a09757365725f6e616d651a086170705f6e616d651a0d757365725f7072696f726974791a07726574726965731a116c6173745f72657472795f726561736f6e1a0870726f626c656d731a066361757365731a1273746d745f657865637574696f6e5f6964731a0d6370755f73716c5f6e616e6f731a0f6c6173745f6572726f725f636f64651a067374617475731a0f636f6e74656e74696f6e5f74696d651a0f636f6e74656e74696f6e5f696e666f1a0764657461696c731a0763726561746564200120022003200420052006200720082009200a200b200c200d200e200f20102011201220132014201520162800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ca8a89","value":"030a891e0a1c73746174656d656e745f657865637574696f6e5f696e7369676874731842200128013a00422f0a0a73657373696f6e5f696410011a0c0807100018003000501960002000300068007000780080010088010098010042340a0e7472616e73616374696f6e5f696410021a0d080e100018003000508617600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c0808100018003000501160002000300068007000780080010088010098010042310a0c73746174656d656e745f696410041a0c08071000180030005019600020003000680070007800800100880100980100423d0a1873746174656d656e745f66696e6765727072696e745f696410051a0c08081000180030005011600020003000680070007800800100880100980100422c0a0770726f626c656d10061a0c08011040180030005014600020013000680070007800800100880100980100423c0a0663617573657310071a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100422a0a05717565727910081a0c08071000180030005019600020013000680070007800800100880100980100422b0a0673746174757310091a0c0801104018003000501460002001300068007000780080010088010098010042300a0a73746172745f74696d65100a1a0d080910001800300050a009600020013000680070007800800100880100980100422e0a08656e645f74696d65100b1a0d080910001800300050a009600020013000680070007800800100880100980100422e0a0966756c6c5f7363616e100c1a0c08001000180030005010600020013000680070007800800100880100980100422e0a09757365725f6e616d65100d1a0c08071000180030005019600020013000680070007800800100880100980100422d0a086170705f6e616d65100e1a0c0807100018003000501960002001300068007000780080010088010098010042320a0d757365725f7072696f72697479100f1a0c0807100018003000501960002001300068007000780080010088010098010042320a0d64617461626173655f6e616d6510101a0c08071000180030005019600020013000680070007800800100880100980100422e0a09706c616e5f6769737410111a0c08071000180030005019600020013000680070007800800100880100980100422c0a077265747269657310121a0c0801104018003000501460002001300068007000780080010088010098010042360a116c6173745f72657472795f726561736f6e10131a0c0807100018003000501960002001300068007000780080010088010098010042480a12657865637574696f6e5f6e6f64655f69647310141a1d080f104018003000380150f8075a0c080110401800300050146000600020013000680070007800800100880100980100424b0a15696e6465785f7265636f6d6d656e646174696f6e7310151a1d080f100018003000380750f1075a0c08071000180030005019600060002001300068007000780080010088010098010042310a0c696d706c696369745f74786e10161a0c0800100018003000501060002001300068007000780080010088010098010042320a0d6370755f73716c5f6e616e6f7310171a0c08011040180030005014600020013000680070007800800100880100980100422f0a0a6572726f725f636f646510181a0c08071000180030005019600020013000680070007800800100880100980100423b0a0f636f6e74656e74696f6e5f74696d6510191a13080610001800300050a20960006a04080010002001300068007000780080010088010098010042350a0f636f6e74656e74696f6e5f696e666f101a1a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0764657461696c73101b1a0d081210001800300050da1d60002001300068007000780080010088010098010042420a0763726561746564101c1a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042a0010a2a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136101d1a0c080110201800300050176000200030015a4f6d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f627974657328656e645f74696d652c2073746172745f74696d652929292c2031363a3a3a494e543829680070007800800101880100980100481e529a040a077072696d61727910011801220c73746174656d656e745f6964220e7472616e73616374696f6e5f69642a0a73657373696f6e5f69642a1a7472616e73616374696f6e5f66696e6765727072696e745f69642a1873746174656d656e745f66696e6765727072696e745f69642a0770726f626c656d2a066361757365732a0571756572792a067374617475732a0a73746172745f74696d652a08656e645f74696d652a0966756c6c5f7363616e2a09757365725f6e616d652a086170705f6e616d652a0d757365725f7072696f726974792a0d64617461626173655f6e616d652a09706c616e5f676973742a07726574726965732a116c6173745f72657472795f726561736f6e2a12657865637574696f6e5f6e6f64655f6964732a15696e6465785f7265636f6d6d656e646174696f6e732a0c696d706c696369745f74786e2a0d6370755f73716c5f6e616e6f732a0a6572726f725f636f64652a0f636f6e74656e74696f6e5f74696d652a0f636f6e74656e74696f6e5f696e666f2a0764657461696c732a076372656174656430043002400040004a10080010001a00200028003000380040005a007001700370057006700770087009700a700b700c700d700e700f7010701170127013701470157016701770187019701a701b701c7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005a7c0a127472616e73616374696f6e5f69645f69647810021800220e7472616e73616374696f6e5f69643002380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab4010a1e7472616e73616374696f6e5f66696e6765727072696e745f69645f69647810031800221a7472616e73616374696f6e5f66696e6765727072696e745f6964220a73746172745f74696d652208656e645f74696d653003300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005ab0010a1c73746174656d656e745f66696e6765727072696e745f69645f69647810041800221873746174656d656e745f66696e6765727072696e745f6964220a73746172745f74696d652208656e645f74696d653005300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005af4010a0e74696d655f72616e67655f69647810051800222a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f3136220a73746172745f74696d652208656e645f74696d65301d300a300b380438024000400140014a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a201460801122a637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313618102208656e645f74696d65220a73746172745f74696d65a80100b20100ba0100c00100c80100d00100e00100e901000000000000000060066a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20193020ad401637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291230636865636b5f637264625f696e7465726e616c5f656e645f74696d655f73746172745f74696d655f73686172645f31361800281d300038014002b201c8030a077072696d61727910001a0a73657373696f6e5f69641a0e7472616e73616374696f6e5f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a0c73746174656d656e745f69641a1873746174656d656e745f66696e6765727072696e745f69641a0770726f626c656d1a066361757365731a0571756572791a067374617475731a0a73746172745f74696d651a08656e645f74696d651a0966756c6c5f7363616e1a09757365725f6e616d651a086170705f6e616d651a0d757365725f7072696f726974791a0d64617461626173655f6e616d651a09706c616e5f676973741a07726574726965731a116c6173745f72657472795f726561736f6e1a12657865637574696f6e5f6e6f64655f6964731a15696e6465785f7265636f6d6d656e646174696f6e731a0c696d706c696369745f74786e1a0d6370755f73716c5f6e616e6f731a0a6572726f725f636f64651a0f636f6e74656e74696f6e5f74696d651a0f636f6e74656e74696f6e5f696e666f1a0764657461696c731a0763726561746564200120022003200420052006200720082009200a200b200c200d200e200f2010201120122013201420152016201720182019201a201b201c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89cb8a89","value":"030ad61c0a0e7461626c655f6d657461646174611843200128013a00422a0a0564625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a087461626c655f696410021a0c08011040180030005014600020003000680070007800800100880100980100422c0a0764625f6e616d6510031a0c0807100018003000501960002000300068007000780080010088010098010042300a0b736368656d615f6e616d6510041a0c08071000180030005019600020003000680070007800800100880100980100422f0a0a7461626c655f6e616d6510051a0c0807100018003000501960002000300068007000780080010088010098010042320a0d746f74616c5f636f6c756d6e7310061a0c0801104018003000501460002000300068007000780080010088010098010042320a0d746f74616c5f696e646578657310071a0c08011040180030005014600020003000680070007800800100880100980100423f0a0973746f72655f69647310081a1d080f104018003000380150f8075a0c080110401800300050146000600020003000680070007800800100880100980100423b0a167265706c69636174696f6e5f73697a655f627974657310091a0c0801104018003000501460002000300068007000780080010088010098010042310a0c746f74616c5f72616e676573100a1a0c08011040180030005014600020003000680070007800800100880100980100423a0a15746f74616c5f6c6976655f646174615f6279746573100b1a0c0801104018003000501460002000300068007000780080010088010098010042350a10746f74616c5f646174615f6279746573100c1a0c0801104018003000501460002000300068007000780080010088010098010042340a0e706572635f6c6976655f64617461100d1a0d080210401800300050bd0560002000300068007000780080010088010098010042360a116c6173745f7570646174655f6572726f72100e1a0c0807100018003000501960002001300068007000780080010088010098010042470a0c6c6173745f75706461746564100f1a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422f0a0a7461626c655f7479706510101a0c08071000180030005019600020003000680070007800800100880100980100422d0a0764657461696c7310111a0d081210001800300050da1d60002000300068007000780080010088010098010042a4010a2c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313610121a0c080110201800300050176000200030015a516d6f6428666e763332286d643528637264625f696e7465726e616c2e646174756d735f746f5f6279746573287461626c655f69642c206c6173745f757064617465642929292c2031363a3a3a494e543829680070007800800101880100980100481352f2020a077072696d61727910011801220564625f696422087461626c655f69642a0764625f6e616d652a0b736368656d615f6e616d652a0a7461626c655f6e616d652a0d746f74616c5f636f6c756d6e732a0d746f74616c5f696e64657865732a0973746f72655f6964732a167265706c69636174696f6e5f73697a655f62797465732a0c746f74616c5f72616e6765732a15746f74616c5f6c6976655f646174615f62797465732a10746f74616c5f646174615f62797465732a0e706572635f6c6976655f646174612a116c6173745f7570646174655f6572726f722a0c6c6173745f757064617465642a0a7461626c655f747970652a0764657461696c7330013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c700d700e700f701070117a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e90100000000000000005aa3010a237265706c69636174696f6e5f73697a655f62797465735f7461626c655f69645f6964781002180022167265706c69636174696f6e5f73697a655f627974657322087461626c655f6964300930023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a8f010a19746f74616c5f72616e6765735f7461626c655f69645f69647810031800220c746f74616c5f72616e67657322087461626c655f6964300a30023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a91010a1a746f74616c5f636f6c756d6e735f7461626c655f69645f69647810041800220d746f74616c5f636f6c756d6e7322087461626c655f6964300630023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a91010a1a746f74616c5f696e64657865735f7461626c655f69645f69647810051800220d746f74616c5f696e646578657322087461626c655f6964300730023801400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a8f010a15706572635f6c6976655f646174615f69645f69647810061800220e706572635f6c6976655f6461746122087461626c655f6964300d30023801400140004a10080010001a00200028003000380040005a00680d7a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005afc010a106c6173745f757064617465645f69647810071800222c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f3136220c6c6173745f7570646174656422087461626c655f69643012300f300238014000400140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a2014a0801122c637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f31361810220c6c6173745f7570646174656422087461626c655f6964a80100b20100ba0100c00100c80100d00100e00100e90100000000000000005a730a0b64625f6e616d655f67696e10081800220764625f6e616d6530033801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a790a0e7461626c655f6e616d655f67696e10091800220a7461626c655f6e616d6530053801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a7b0a0f736368656d615f6e616d655f67696e100a1800220b736368656d615f6e616d6530043801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80101e00100e90100000000000000005a770a0d73746f72655f6964735f67696e100b1800220973746f72655f69647330083801380240004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e00100e9010000000000000000600c6a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100a20197020ad601637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313620494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e54382c20383a3a3a494e54382c20393a3a3a494e54382c2031303a3a3a494e54382c2031313a3a3a494e54382c2031323a3a3a494e54382c2031333a3a3a494e54382c2031343a3a3a494e54382c2031353a3a3a494e5438291232636865636b5f637264625f696e7465726e616c5f6c6173745f757064617465645f7461626c655f69645f73686172645f313618002812300038014002b201a0020a077072696d61727910001a0564625f69641a087461626c655f69641a0764625f6e616d651a0b736368656d615f6e616d651a0a7461626c655f6e616d651a0d746f74616c5f636f6c756d6e731a0d746f74616c5f696e64657865731a0973746f72655f6964731a167265706c69636174696f6e5f73697a655f62797465731a0c746f74616c5f72616e6765731a15746f74616c5f6c6976655f646174615f62797465731a10746f74616c5f646174615f62797465731a0e706572635f6c6976655f646174611a116c6173745f7570646174655f6572726f721a0c6c6173745f757064617465641a0a7461626c655f747970651a0764657461696c73200120022003200420052006200720082009200a200b200c200d200e200f201020112800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89cc8a89","value":"030ab0040a0c6a6f625f70726f67726573731844200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422e0a086672616374696f6e10031a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a087265736f6c76656410041a0d080310001800300050a40d6000200130006800700078008001008801009801004805528c010a077072696d6172791001180122066a6f625f696422077772697474656e2a086672616374696f6e2a087265736f6c76656430013002400040014a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2013a0a077072696d61727910001a066a6f625f69641a077772697474656e1a086672616374696f6e1a087265736f6c76656420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89cd8a89","value":"030ab8040a146a6f625f70726f67726573735f686973746f72791845200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422e0a086672616374696f6e10031a0d080210401800300050bd05600020013000680070007800800100880100980100422e0a087265736f6c76656410041a0d080310001800300050a40d6000200130006800700078008001008801009801004805528c010a077072696d6172791001180122066a6f625f696422077772697474656e2a086672616374696f6e2a087265736f6c76656430013002400040014a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2013a0a077072696d61727910001a066a6f625f69641a077772697474656e1a086672616374696f6e1a087265736f6c76656420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89ce8a89","value":"030ade030a0a6a6f625f7374617475731846200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422b0a0673746174757310031a0c080710001800300050196000200030006800700078008001008801009801004804527e0a077072696d6172791001180122066a6f625f696422077772697474656e2a0673746174757330013002400040014a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b2012c0a077072696d61727910001a066a6f625f69641a077772697474656e1a067374617475732001200220032803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89cf8a89","value":"030aa0040a0b6a6f625f6d6573736167651847200128013a00422b0a066a6f625f696410011a0c0801104018003000501460002000300068007000780080010088010098010042420a077772697474656e10021a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a300068007000780080010088010098010042290a046b696e6410031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076d65737361676510041a0c0807100018003000501960002000300068007000780080010088010098010048055289010a077072696d6172791001180122066a6f625f696422077772697474656e22046b696e642a076d6573736167653001300230034000400140004a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651803800101880103980100b201350a077072696d61727910001a066a6f625f69641a077772697474656e1a046b696e641a076d65737361676520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} +,{"key":"8b89d08a89","value":"030ab4060a1570726570617265645f7472616e73616374696f6e731848200128013a00422e0a09676c6f62616c5f696410011a0c0807100018003000501960002000300068007000780080010088010098010042340a0e7472616e73616374696f6e5f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042340a0f7472616e73616374696f6e5f6b657910031a0c0808100018003000501160002001300068007000780080010088010098010042430a08707265706172656410041a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a056f776e657210051a0c08071000180030005019600020003000680070007800800100880100980100422d0a08646174616261736510061a0c08071000180030005019600020003000680070007800800100880100980100422e0a0968657572697374696310071a0c08071000180030005019600020013000680070007800800100880100980100480852bd010a077072696d617279100118012209676c6f62616c5f69642a0e7472616e73616374696f6e5f69642a0f7472616e73616374696f6e5f6b65792a0870726570617265642a056f776e65722a0864617461626173652a09686575726973746963300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e00100e901000000000000000060026a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651803800101880103980100b2016d0a077072696d61727910001a09676c6f62616c5f69641a0e7472616e73616374696f6e5f69641a0f7472616e73616374696f6e5f6b65791a0870726570617265641a056f776e65721a0864617461626173651a0968657572697374696320012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300d00300d80300e00300f80300880400980400"} ,{"key":"8d89888a89","value":"031080808040188080808002220308c0702803500058007801"} ,{"key":"8f898888","value":"01c801"} ,{"key":"90898988","value":"0a2a160c080110001a0020002a004200160673797374656d13021304"} diff --git a/pkg/sql/catalog/descpb/structured.go b/pkg/sql/catalog/descpb/structured.go index 80713dae9671..ac5ce51a8298 100644 --- a/pkg/sql/catalog/descpb/structured.go +++ b/pkg/sql/catalog/descpb/structured.go @@ -72,6 +72,9 @@ type ConstraintID = catid.ConstraintID // TriggerID is a custom type for TableDescriptor trigger IDs. type TriggerID = catid.TriggerID +// PolicyID is a custom type for TableDescriptor policy IDs. +type PolicyID = catid.PolicyID + // DescriptorVersion is a custom type for TableDescriptor Versions. type DescriptorVersion uint64 diff --git a/pkg/sql/catalog/descpb/structured.proto b/pkg/sql/catalog/descpb/structured.proto index 520df3ab7205..102aad66c814 100644 --- a/pkg/sql/catalog/descpb/structured.proto +++ b/pkg/sql/catalog/descpb/structured.proto @@ -695,6 +695,19 @@ message MaterializedViewRefresh { optional bool should_backfill = 4 [(gogoproto.nullable) = false]; } +// PolicyDescriptor is a representation of a row-level security policy. It's +// stored in the TableDescriptor. +message PolicyDescriptor { + option (gogoproto.equal) = true; + + // ID is used within the table descriptor to uniquely identify policies. + optional uint32 id = 1 [(gogoproto.customname) = "ID", + (gogoproto.casttype) = "PolicyID", (gogoproto.nullable) = false]; + + // The name of the policy. Unique within a table, and cannot be qualified. + optional string name = 2 [(gogoproto.nullable) = false]; +} + // A DescriptorMutation represents a column or an index that // has either been added or dropped and hasn't yet transitioned // into a stable state: completely backfilled and visible, or @@ -1296,7 +1309,14 @@ message TableDescriptor { optional uint32 next_trigger_id = 65 [(gogoproto.nullable) = false, (gogoproto.customname) = "NextTriggerID", (gogoproto.casttype) = "TriggerID"]; - // Next ID: 66 + // Policies are the row-level security policies that are defined on this table. + repeated PolicyDescriptor policies = 66 [(gogoproto.nullable) = false]; + + // NextPolicyID is the ID for the next policy created. + optional uint32 next_policy_id = 67 [(gogoproto.nullable) = false, + (gogoproto.customname) = "NextPolicyID", (gogoproto.casttype) = "PolicyID"]; + + // Next ID: 68 } // ExternalRowData indicates that the row data for this object is stored outside diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 7084e96b3d3a..c27b9161c1d0 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -830,6 +830,11 @@ type TableDescriptor interface { // GetNextTriggerID returns the next unused trigger ID for this table. // Trigger IDs are unique per table, but not unique globally. GetNextTriggerID() descpb.TriggerID + // GetPolicies returns a slice with all policies defined on the table. + GetPolicies() []descpb.PolicyDescriptor + // GetNextPolicyID returns the next unused policy ID for this table. + // Policy IDs are unique per table. + GetNextPolicyID() descpb.PolicyID } // MutableTableDescriptor is both a MutableDescriptor and a TableDescriptor. diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index 914a7d7662b8..2efdda4f826c 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -1179,6 +1179,19 @@ func FindTriggerByID(tbl TableDescriptor, id descpb.TriggerID) *descpb.TriggerDe return nil } +// FindPolicyByID traverses the slice returned by the GetPolicies method on the +// table descriptor and returns the first policy that matches the desired ID, +// or nil if none was found. +func FindPolicyByID(tbl TableDescriptor, id descpb.PolicyID) *descpb.PolicyDescriptor { + policies := tbl.GetPolicies() + for i := range policies { + if policies[i].ID == id { + return &policies[i] + } + } + return nil +} + // FindFamilyByID traverses the family descriptors on the table descriptor // and returns the first column family with the desired ID, or nil if none was // found. diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index af3d9f414835..10acefd5e29f 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -469,6 +469,12 @@ func ConstraintNamePlaceholder(id descpb.ConstraintID) string { return fmt.Sprintf("crdb_internal_constraint_%d_name_placeholder", id) } +// PolicyNamePlaceholder constructs a placeholder name for a policy based +// on its id. +func PolicyNamePlaceholder(id descpb.PolicyID) string { + return fmt.Sprintf("crdb_internal_policy_%d_name_placeholder", id) +} + // RenameColumnInTable will rename the column in tableDesc from oldName to // newName, including in expressions as well as shard columns. // The function is recursive because of this, but there should only be one level diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 4b1e6252ed23..0ff7d7fb9e8a 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -899,6 +899,7 @@ func (desc *wrapper) ValidateSelf(vea catalog.ValidationErrorAccumulator) { desc.validateUniqueWithoutIndexConstraints(columnsByID), desc.validateTableIndexes(columnsByID, vea.IsActive), desc.validatePartitioning(), + desc.validatePolicies(), } hasErrs := false for _, err := range newErrs { @@ -2049,6 +2050,41 @@ func (desc *wrapper) validatePartitioning() error { }) } +func (desc *wrapper) validatePolicies() error { + if !desc.IsTable() { + return nil + } + policies := desc.GetPolicies() + names := make(map[string]descpb.PolicyID, len(policies)) + idToName := make(map[descpb.PolicyID]string, len(policies)) + for _, p := range policies { + if p.ID == 0 { + return errors.AssertionFailedf( + "policy ID was missing for policy %q", + p.Name) + } else if p.ID >= desc.NextPolicyID { + return errors.AssertionFailedf( + "policy %q has ID %d, which is not less than the NextPolicyID value %d for the table", + p.Name, p.ID, desc.NextPolicyID) + } + if p.Name == "" { + return pgerror.Newf(pgcode.Syntax, "empty policy name") + } + if otherID, found := names[p.Name]; found && p.ID != otherID { + return pgerror.Newf(pgcode.DuplicateObject, + "duplicate policy name: %q", p.Name) + } + names[p.Name] = p.ID + if other, found := idToName[p.ID]; found { + return pgerror.Newf(pgcode.DuplicateObject, + "policy ID %d in policy %q already in use by %q", + p.ID, p.Name, other) + } + idToName[p.ID] = p.Name + } + return nil +} + // validateAutoStatsSettings validates that any new settings in // catpb.AutoStatsSettings hold a valid value. func (desc *wrapper) validateAutoStatsSettings(vea catalog.ValidationErrorAccumulator) { diff --git a/pkg/sql/catalog/tabledesc/validate_test.go b/pkg/sql/catalog/tabledesc/validate_test.go index d6276615484c..ac48cdd76d79 100644 --- a/pkg/sql/catalog/tabledesc/validate_test.go +++ b/pkg/sql/catalog/tabledesc/validate_test.go @@ -143,6 +143,8 @@ var validationMap = []struct { "ReplicatedPCRVersion": {status: thisFieldReferencesNoObjects}, "Triggers": {status: iSolemnlySwearThisFieldIsValidated}, "NextTriggerID": {status: thisFieldReferencesNoObjects}, + "Policies": {status: iSolemnlySwearThisFieldIsValidated}, + "NextPolicyID": {status: iSolemnlySwearThisFieldIsValidated}, }, }, { @@ -3078,6 +3080,69 @@ func TestValidateTableDesc(t *testing.T) { }, NextColumnID: 2, }}, + {err: `policy ID was missing for policy "pol"`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 1 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 0, + Name: "pol", + }, + } + }), + }, + {err: `empty policy name`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 2 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 1, + Name: "", + }, + } + }), + }, + {err: `duplicate policy name: "pol"`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 3 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 1, + Name: "pol", + }, + { + ID: 2, + Name: "pol", + }, + } + }), + }, + {err: `policy ID 10 in policy "pol_new" already in use by "pol_old"`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 11 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 10, + Name: "pol_old", + }, + { + ID: 10, + Name: "pol_new", + }, + } + }), + }, + {err: `policy "pol" has ID 20, which is not less than the NextPolicyID value 5 for the table`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 5 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 20, + Name: "pol", + }, + } + }), + }, } for i, d := range testData { diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_security b/pkg/sql/logictest/testdata/logic_test/row_level_security index a8a94c71a12b..946fb57687b1 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_security +++ b/pkg/sql/logictest/testdata/logic_test/row_level_security @@ -1,3 +1,5 @@ +# LogicTest: !local-legacy-schema-changer !local-mixed-24.3 + statement ok SET enable_row_level_security = on; @@ -5,3 +7,104 @@ query T show session enable_row_level_security; ---- on + +subtest legacy_fail + +let $use_decl_sc +SHOW use_declarative_schema_changer + +statement ok +SET use_declarative_schema_changer = 'off'; + +statement ok +CREATE TABLE legacy(); + +statement error pq: CREATE POLICY is only implemented in the declarative schema changer +CREATE POLICY p1 on legacy; + +statement ok +DROP TABLE legacy; + +statement ok +SET use_declarative_schema_changer = $use_decl_sc; + +subtest create_drop_sanity + +statement ok +CREATE TABLE sanity1(); + +statement ok +CREATE POLICY p1 on sanity1 USING (true); + +statement error pq: policy with name "p1" already exists on table "sanity1" +CREATE POLICY p1 on sanity1 WITH CHECK (true); + +statement ok +CREATE POLICY p2 on sanity1 AS PERMISSIVE WITH CHECK (true); + +statement notice NOTICE: relation "nonexist" does not exist, skipping +DROP POLICY IF EXISTS notthere on nonexist; + +statement notice NOTICE: policy "notthere" for relation "sanity1" does not exist, skipping +DROP POLICY IF EXISTS notthere on sanity1; + +statement error pq: policy "notthere" for table "sanity1" does not exist +DROP POLICY notthere on sanity1; + +statement ok +DROP POLICY p1 on sanity1; + +statement ok +DROP POLICY p2 on sanity1; + +statement ok +CREATE POLICY newp1 on sanity1 AS PERMISSIVE USING (true) WITH CHECK (true); + +statement ok +DROP TABLE sanity1; + +subtest explicit_txn + +statement ok +CREATE TABLE explicit1(); + +let $use_decl_sc +SHOW use_declarative_schema_changer + +statement ok +SET use_declarative_schema_changer = 'unsafe_always'; + +statement ok +BEGIN TRANSACTION ISOLATION LEVEL SERIALIZABLE; + +statement ok +CREATE POLICY p1 on explicit1; + +statement ok +DROP POLICY p1 on explicit1; + +statement ok +CREATE POLICY p1 on explicit1 AS PERMISSIVE USING (false); + +statement ok +COMMIT; + +statement ok +BEGIN TRANSACTION ISOLATION LEVEL SERIALIZABLE; + +statement ok +DROP POLICY p1 on explicit1; + +statement ok +CREATE POLICY p1 on explicit1 USING (false) WITH CHECK (true); + +statement ok +COMMIT; + +statement ok +DROP TABLE explicit1; + +statement ok +SET use_declarative_schema_changer = $use_decl_sc; + +subtest end diff --git a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go index f5639769ae86..808c55191f07 100644 --- a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go +++ b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go @@ -1564,13 +1564,6 @@ func TestLogic_routine_schema_change( runLogicTest(t, "routine_schema_change") } -func TestLogic_row_level_security( - t *testing.T, -) { - defer leaktest.AfterTest(t)() - runLogicTest(t, "row_level_security") -} - func TestLogic_row_level_ttl( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-mixed-24.3/generated_test.go b/pkg/sql/logictest/tests/local-mixed-24.3/generated_test.go index 8f9c07e9973e..787f0c3265c6 100644 --- a/pkg/sql/logictest/tests/local-mixed-24.3/generated_test.go +++ b/pkg/sql/logictest/tests/local-mixed-24.3/generated_test.go @@ -1585,13 +1585,6 @@ func TestLogic_routine_schema_change( runLogicTest(t, "routine_schema_change") } -func TestLogic_row_level_security( - t *testing.T, -) { - defer leaktest.AfterTest(t)() - runLogicTest(t, "row_level_security") -} - func TestLogic_row_level_ttl( t *testing.T, ) { diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace b/pkg/sql/opt/exec/execbuilder/testdata/show_trace index f40e26584db6..28ab72c80322 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace @@ -64,7 +64,7 @@ SET tracing = on,kv,results; CREATE TABLE t.kv(k INT PRIMARY KEY, v INT, FAMILY query TT $trace_query ---- -create table CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:2 privileges: users: owner_proto:"root" version:3 > next_mutation_id:1 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:2 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 > +create table CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:2 privileges: users: owner_proto:"root" version:3 > next_mutation_id:1 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:2 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 next_policy_id:0 > create table CPut /NamespaceTable/30/1/106/107/"kv"/4/1 -> 108 sql query rows affected: 0 @@ -80,7 +80,7 @@ SET tracing = on,kv,results; CREATE UNIQUE INDEX woo ON t.kv(v); SET tracing = o query TT $trace_query ---- -create index CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:4 privileges: users: owner_proto:"root" version:3 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 not_visible:false invisibility:0 > state:BACKFILLING direction:ADD mutation_id:1 rollback:false > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:true created_at_nanos:... constraint_id:3 not_visible:false invisibility:0 > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 > +create index CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:4 privileges: users: owner_proto:"root" version:3 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 not_visible:false invisibility:0 > state:BACKFILLING direction:ADD mutation_id:1 rollback:false > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:true created_at_nanos:... constraint_id:3 not_visible:false invisibility:0 > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 next_policy_id:0 > sql query rows affected: 0 statement ok @@ -126,7 +126,7 @@ SET tracing = off query TT $trace_query ---- -create table CPut /Table/3/1/109/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:2 privileges: users: owner_proto:"root" version:3 > next_mutation_id:1 format_version:3 state:ADD offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:2 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 > +create table CPut /Table/3/1/109/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:2 privileges: users: owner_proto:"root" version:3 > next_mutation_id:1 format_version:3 state:ADD offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:2 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 next_policy_id:0 > create table CPut /NamespaceTable/30/1/106/107/"kv2"/4/1 -> 109 sql query rows affected: 0 @@ -165,7 +165,7 @@ query TT $trace_query ---- sql query rows affected: 0 -commit sql txn CPut /Table/3/1/109/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:2 privileges: users: owner_proto:"root" version:3 > next_mutation_id:1 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"DECIMAL" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"OID" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT4" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"DECIMAL" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:DROPPED current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 target_ranks:5 target_ranks:6 target_ranks:7 target_ranks:8 target_ranks:9 target_ranks:10 target_ranks:11 target_ranks:12 target_ranks:13 target_ranks:14 target_ranks:15 target_ranks:16 target_ranks:17 target_ranks:18 target_ranks:19 target_ranks:20 target_ranks:21 target_ranks:22 target_ranks:23 target_ranks:24 target_ranks:25 target_ranks:26 target_ranks:27 target_ranks:28 target_ranks:29 target_ranks:30 target_ranks:31 target_ranks:32 target_ranks:33 target_ranks:34 target_ranks:35 target_ranks:36 relevant_statements: > authorization: name_mapping: > drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:2 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 > +commit sql txn CPut /Table/3/1/109/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:2 privileges: users: owner_proto:"root" version:3 > next_mutation_id:1 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"DECIMAL" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"OID" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT4" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"DECIMAL" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:DROPPED current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 target_ranks:5 target_ranks:6 target_ranks:7 target_ranks:8 target_ranks:9 target_ranks:10 target_ranks:11 target_ranks:12 target_ranks:13 target_ranks:14 target_ranks:15 target_ranks:16 target_ranks:17 target_ranks:18 target_ranks:19 target_ranks:20 target_ranks:21 target_ranks:22 target_ranks:23 target_ranks:24 target_ranks:25 target_ranks:26 target_ranks:27 target_ranks:28 target_ranks:29 target_ranks:30 target_ranks:31 target_ranks:32 target_ranks:33 target_ranks:34 target_ranks:35 target_ranks:36 relevant_statements: > authorization: name_mapping: > drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:2 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 next_policy_id:0 > commit sql txn Del /NamespaceTable/30/1/106/107/"kv2"/4/1 statement ok @@ -194,7 +194,7 @@ query TT $trace_query ---- sql query rows affected: 0 -commit sql txn CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:4 privileges: users: owner_proto:"root" version:3 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 not_visible:false invisibility:0 > state:WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:PUBLIC > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata:<> target_status:PUBLIC > current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:VALIDATED current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 target_ranks:5 target_ranks:6 relevant_statements: > authorization: name_mapping: columns: columns: columns: columns: columns: families: indexes: > > drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 > +commit sql txn CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:4 privileges: users: owner_proto:"root" version:3 > mutations: interleave:<> partitioning: type:FORWARD created_explicitly:true encoding_type:0 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:2 not_visible:false invisibility:0 > state:WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:PUBLIC > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata:<> target_status:PUBLIC > current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:VALIDATED current_statuses:PUBLIC current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 target_ranks:5 target_ranks:6 relevant_statements: > authorization: name_mapping: columns: columns: columns: columns: columns: families: indexes: > > drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 next_policy_id:0 > statement ok SET tracing = on,kv,results; DROP TABLE t.kv @@ -206,7 +206,7 @@ query TT $trace_query ---- sql query rows affected: 0 -commit sql txn CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:4 privileges: users: owner_proto:"root" version:3 > next_mutation_id:2 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"DECIMAL" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"OID" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT4" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"DECIMAL" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:DROPPED current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 target_ranks:5 target_ranks:6 target_ranks:7 target_ranks:8 target_ranks:9 target_ranks:10 target_ranks:11 target_ranks:12 target_ranks:13 target_ranks:14 target_ranks:15 target_ranks:16 target_ranks:17 target_ranks:18 target_ranks:19 target_ranks:20 target_ranks:21 target_ranks:22 target_ranks:23 target_ranks:24 target_ranks:25 target_ranks:26 target_ranks:27 target_ranks:28 target_ranks:29 target_ranks:30 target_ranks:31 relevant_statements: > authorization: name_mapping: > drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 > +commit sql txn CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false invisibility:0 > next_index_id:4 privileges: users: owner_proto:"root" version:3 > next_mutation_id:2 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT8" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"DECIMAL" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"OID" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"INT4" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: type_name:"DECIMAL" > is_nullable:true element_creation_metadata: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:DROPPED current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 target_ranks:5 target_ranks:6 target_ranks:7 target_ranks:8 target_ranks:9 target_ranks:10 target_ranks:11 target_ranks:12 target_ranks:13 target_ranks:14 target_ranks:15 target_ranks:16 target_ranks:17 target_ranks:18 target_ranks:19 target_ranks:20 target_ranks:21 target_ranks:22 target_ranks:23 target_ranks:24 target_ranks:25 target_ranks:26 target_ranks:27 target_ranks:28 target_ranks:29 target_ranks:30 target_ranks:31 relevant_statements: > authorization: name_mapping: > drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 schema_locked:false import_epoch:0 import_type:IMPORT_WITH_START_TIME_ONLY replicated_pcr_version:0 next_trigger_id:0 next_policy_id:0 > commit sql txn Del /NamespaceTable/30/1/106/107/"kv"/4/1 # Check that session tracing does not inhibit the fast path for inserts & diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 93cc2ec5c619..26da6685c608 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -4957,7 +4957,7 @@ alter_policy_stmt: /* SKIP DOC */ $$.val = &tree.AlterPolicy{ PolicyName: tree.Name($3), - TableName: $5.unresolvedObjectName().ToTableName(), + TableName: $5.unresolvedObjectName(), NewPolicyName: tree.Name($8), } } @@ -4966,7 +4966,7 @@ alter_policy_stmt: /* SKIP DOC */ $$.val = &tree.AlterPolicy{ PolicyName: tree.Name($3), - TableName: $5.unresolvedObjectName().ToTableName(), + TableName: $5.unresolvedObjectName(), Roles: $6.roleSpecList(), Exprs: $7.policyExpressions(), } @@ -4990,7 +4990,7 @@ create_policy_stmt: /* SKIP DOC */ $$.val = &tree.CreatePolicy{ PolicyName: tree.Name($3), - TableName: $5.unresolvedObjectName().ToTableName(), + TableName: $5.unresolvedObjectName(), Type: $6.policyType(), Cmd: $7.policyCommand(), Roles: $8.roleSpecList(), diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index 831efbe47194..511a754eae85 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -517,6 +517,35 @@ func (b *builderState) NextTableTriggerID(tableID catid.DescID) (ret catid.Trigg return ret } +// NextTablePolicyID implements the scbuildstmt.TableHelpers interface. +func (b *builderState) NextTablePolicyID(tableID catid.DescID) (ret catid.PolicyID) { + { + b.ensureDescriptor(tableID) + desc := b.descCache[tableID].desc + tbl, ok := desc.(catalog.TableDescriptor) + if !ok { + panic(errors.AssertionFailedf("Expected table descriptor for ID %d, instead got %s", + desc.GetID(), desc.DescriptorType())) + } + ret = tbl.GetNextPolicyID() + if ret == 0 { + ret = 1 + } + } + // Consult all present element in case they have a larger PolicyID field. + b.QueryByID(tableID).ForEach(func( + _ scpb.Status, _ scpb.TargetStatus, e scpb.Element, + ) { + v, _ := screl.Schema.GetAttribute(screl.PolicyID, e) + if id, ok := v.(catid.PolicyID); ok { + if id < catid.PolicyID(scbuildstmt.TableTentativeIdsStart) && id >= ret { + ret = id + 1 + } + } + }) + return ret +} + // NextTableTentativeIndexID implements the scbuildstmt.TableHelpers interface. func (b *builderState) NextTableTentativeIndexID(tableID catid.DescID) (ret catid.IndexID) { ret = catid.IndexID(scbuildstmt.TableTentativeIdsStart) @@ -1448,6 +1477,30 @@ func (b *builderState) ResolveTrigger( }) } +func (b *builderState) ResolvePolicy( + tableID catid.DescID, policyName tree.Name, p scbuildstmt.ResolveParams, +) scbuildstmt.ElementResultSet { + b.ensureDescriptor(tableID) + tbl := b.descCache[tableID].desc.(catalog.TableDescriptor) + elems := b.QueryByID(tbl.GetID()) + var policyID catid.PolicyID + elems.ForEach(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) { + if t, ok := e.(*scpb.PolicyName); ok && t.Name == string(policyName) { + policyID = t.PolicyID + } + }) + if policyID == 0 { + if p.IsExistenceOptional { + return nil + } + panic(sqlerrors.NewUndefinedPolicyError(string(policyName), tbl.GetName())) + } + return elems.Filter(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) bool { + id, _ := screl.Schema.GetAttribute(screl.PolicyID, e) + return id != nil && id.(catid.PolicyID) == policyID + }) +} + func (b *builderState) newCachedDesc(id descpb.ID) *cachedDesc { return &cachedDesc{ desc: b.readDescriptor(id), diff --git a/pkg/sql/schemachanger/scbuild/builder_test.go b/pkg/sql/schemachanger/scbuild/builder_test.go index 9bc959302ef9..ec1e963ee827 100644 --- a/pkg/sql/schemachanger/scbuild/builder_test.go +++ b/pkg/sql/schemachanger/scbuild/builder_test.go @@ -97,6 +97,7 @@ func TestBuildDataDriven(t *testing.T) { sd.NewSchemaChangerMode = sessiondatapb.UseNewSchemaChangerUnsafeAlways sd.ApplicationName = "" sd.EnableUniqueWithoutIndexConstraints = true + sd.RowLevelSecurityEnabled = true sd.SerialNormalizationMode = localData.SerialNormalizationMode }, ), diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel index b1d7d5e57a11..20dda974429a 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel @@ -105,6 +105,7 @@ go_library( "//pkg/sql/storageparam/indexstorageparam", "//pkg/sql/types", "//pkg/util/encoding", + "//pkg/util/envutil", "//pkg/util/errorutil/unimplemented", "//pkg/util/log/eventpb", "//pkg/util/log/logpb", diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go index 9982c85c5e01..142136e08644 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go @@ -6,11 +6,44 @@ package scbuildstmt 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/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" ) // CreatePolicy implements CREATE POLICY. func CreatePolicy(b BuildCtx, n *tree.CreatePolicy) { - panic(unimplemented.NewWithIssue(136696, "CREATE POLICY is not yet implemented")) + failIfRLSIsNotEnabled(b) + b.IncrementSchemaChangeCreateCounter("policy") + + tableElems := b.ResolveTable(n.TableName, ResolveParams{ + RequiredPrivilege: privilege.CREATE, + }) + panicIfSchemaChangeIsDisallowed(tableElems, n) + tbl := tableElems.FilterTable().MustGetOneElement() + + // Resolve the policy name to make sure one doesn't already exist + policyElems := b.ResolvePolicy(tbl.TableID, n.PolicyName, ResolveParams{ + IsExistenceOptional: true, + RequiredPrivilege: privilege.CREATE, + }) + policyElems.FilterPolicyName().ForEachTarget(func(target scpb.TargetStatus, e *scpb.PolicyName) { + if target == scpb.ToPublic { + panic(pgerror.Newf(pgcode.DuplicateObject, "policy with name %q already exists on table %q", + n.PolicyName, n.TableName.String())) + } + }) + + policyID := b.NextTablePolicyID(tbl.TableID) + b.Add(&scpb.Policy{ + TableID: tbl.TableID, + PolicyID: policyID, + }) + b.Add(&scpb.PolicyName{ + TableID: tbl.TableID, + PolicyID: policyID, + Name: string(n.PolicyName), + }) } diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go index 9924ca01ad98..773ae07fba36 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go @@ -279,6 +279,10 @@ type TableHelpers interface { // added to this table. NextTableTriggerID(tableID catid.DescID) catid.TriggerID + // NextTablePolicyID returns the ID that should be used for any new row-level + // security policies added to this table. + NextTablePolicyID(tableID catid.DescID) catid.PolicyID + // NextTableTentativeIndexID returns the tentative ID, starting from // scbuild.TABLE_TENTATIVE_IDS_START, that should be used for any new index added to // this table. @@ -432,6 +436,9 @@ type NameResolver interface { // ResolveTrigger retrieves a trigger by name and returns its elements. ResolveTrigger(relationID catid.DescID, triggerName tree.Name, p ResolveParams) ElementResultSet + + // ResolvePolicy retrieves a policy by name and returns its elements. + ResolvePolicy(relationID catid.DescID, policyName tree.Name, p ResolveParams) ElementResultSet } // ReferenceProvider provides all referenced objects with in current DDL diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_policy.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_policy.go index 1d8d969e5ac4..a8432ae9ce58 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_policy.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_policy.go @@ -6,11 +6,42 @@ package scbuildstmt import ( + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" ) // DropPolicy implements DROP POLICY. func DropPolicy(b BuildCtx, n *tree.DropPolicy) { - panic(unimplemented.NewWithIssue(136696, "DROP POLICY is not yet implemented")) + failIfRLSIsNotEnabled(b) + noticeSender := b.EvalCtx().ClientNoticeSender + tableElems := b.ResolveTable(n.TableName, ResolveParams{ + IsExistenceOptional: n.IfExists, + RequireOwnership: true, + }) + tbl := tableElems.FilterTable().MustGetZeroOrOneElement() + if tbl == nil { // this can happen IF EXISTS was used with the drop + noticeSender.BufferClientNotice(b, + pgnotice.Newf("relation %q does not exist, skipping", n.TableName.String())) + return + } + panicIfSchemaChangeIsDisallowed(tableElems, n) + + policyElems := b.ResolvePolicy(tbl.TableID, n.PolicyName, ResolveParams{ + IsExistenceOptional: n.IfExists, + }) + policy := policyElems.FilterPolicy().MustGetZeroOrOneElement() + if policy == nil { // this can happen IF EXISTS was used with the drop + noticeSender.BufferClientNotice(b, + pgnotice.Newf("policy %q for relation %q does not exist, skipping", + n.PolicyName, n.TableName.String())) + return + } + policyElems.ForEach(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) { + switch e.(type) { + case *scpb.Policy, *scpb.PolicyName: + b.Drop(e) + } + }) + b.IncrementSchemaChangeDropCounter("policy") } diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go index 7a3ed53bc093..11e3d771e7d8 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go @@ -9,6 +9,7 @@ import ( "fmt" "sort" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" @@ -27,6 +28,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlclustersettings" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/envutil" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -1818,3 +1821,20 @@ func mustRetrievePartitioningFromIndexPartitioning( } return partition } + +// enableRLSEnvVar is true if row-level security is enabled. This override is a +// convenience for dev as it allows you to set an environment variable and not +// have to worry about changing a local setting each time. This should be removed +// once RLS is enabled by default. +var enableRLSEnvVar = envutil.EnvOrDefaultBool("COCKROACH_ENABLE_ROW_LEVEL_SECURITY", false) + +// failIfRLSIsNotEnabled will fail if row-level security is not active +func failIfRLSIsNotEnabled(b BuildCtx) { + if enableRLSEnvVar { + return + } + if !b.SessionData().RowLevelSecurityEnabled || + !b.EvalCtx().Settings.Version.ActiveVersion(b).IsActive(clusterversion.V25_1) { + panic(unimplemented.NewWithIssue(136696, "CREATE POLICY is not yet implemented")) + } +} diff --git a/pkg/sql/schemachanger/scdecomp/decomp.go b/pkg/sql/schemachanger/scdecomp/decomp.go index 885ecb612b3e..c3e5f526b653 100644 --- a/pkg/sql/schemachanger/scdecomp/decomp.go +++ b/pkg/sql/schemachanger/scdecomp/decomp.go @@ -411,6 +411,10 @@ func (w *walkCtx) walkRelation(tbl catalog.TableDescriptor) { for i := range triggers { w.walkTrigger(tbl, &triggers[i]) } + policies := tbl.GetPolicies() + for i := range policies { + w.walkPolicy(tbl, &policies[i]) + } _ = tbl.ForeachDependedOnBy(func(dep *descpb.TableDescriptor_Reference) error { w.backRefs.Add(dep.ID) @@ -870,6 +874,18 @@ func (w *walkCtx) walkTrigger(tbl catalog.TableDescriptor, t *descpb.TriggerDesc }) } +func (w *walkCtx) walkPolicy(tbl catalog.TableDescriptor, p *descpb.PolicyDescriptor) { + w.ev(scpb.Status_PUBLIC, &scpb.Policy{ + TableID: tbl.GetID(), + PolicyID: p.ID, + }) + w.ev(scpb.Status_PUBLIC, &scpb.PolicyName{ + TableID: tbl.GetID(), + PolicyID: p.ID, + Name: p.Name, + }) +} + func (w *walkCtx) walkForeignKeyConstraint( tbl catalog.TableDescriptor, c catalog.ForeignKeyConstraint, ) { diff --git a/pkg/sql/schemachanger/scdeps/sctestutils/sctestutils.go b/pkg/sql/schemachanger/scdeps/sctestutils/sctestutils.go index 8422a2e17176..13def51896e2 100644 --- a/pkg/sql/schemachanger/scdeps/sctestutils/sctestutils.go +++ b/pkg/sql/schemachanger/scdeps/sctestutils/sctestutils.go @@ -87,6 +87,7 @@ func WithBuilderDependenciesFromTestServer( planner.SessionData().NewSchemaChangerMode = sessiondatapb.UseNewSchemaChangerUnsafe planner.SessionData().EnableUniqueWithoutIndexConstraints = true planner.SessionData().AlterColumnTypeGeneralEnabled = true + planner.SessionData().RowLevelSecurityEnabled = true fn(scdeps.NewBuilderDependencies( execCfg.NodeInfo.LogicalClusterID(), execCfg.Codec, diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel index aaa69fab7373..88605ad08c64 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "function.go", "helpers.go", "index.go", + "policy.go", "privileges.go", "range.go", "references.go", diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go index 9df03c66dec5..ed0651151c34 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go @@ -137,6 +137,21 @@ func (i *immediateVisitor) checkOutTrigger( triggerID, tbl.GetName(), tbl.GetID())) } +func (i *immediateVisitor) checkOutPolicy( + ctx context.Context, tableID descpb.ID, policyID catid.PolicyID, +) (*descpb.PolicyDescriptor, error) { + tbl, err := i.checkOutTable(ctx, tableID) + if err != nil { + return nil, err + } + policy := catalog.FindPolicyByID(tbl, policyID) + if policy != nil { + return policy, nil + } + panic(errors.AssertionFailedf("failed to find policy with ID %d in table %q (%d)", + policyID, tbl.GetName(), tbl.GetID())) +} + func mutationStateChange( tbl *tabledesc.Mutable, f MutationSelector, diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/policy.go b/pkg/sql/schemachanger/scexec/scmutationexec/policy.go new file mode 100644 index 000000000000..c10e78b13b72 --- /dev/null +++ b/pkg/sql/schemachanger/scexec/scmutationexec/policy.go @@ -0,0 +1,57 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package scmutationexec + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/errors" +) + +func (i *immediateVisitor) AddPolicy(ctx context.Context, op scop.AddPolicy) error { + tbl, err := i.checkOutTable(ctx, op.Policy.TableID) + if err != nil { + return err + } + if op.Policy.PolicyID >= tbl.NextPolicyID { + tbl.NextPolicyID = op.Policy.PolicyID + 1 + } + tbl.Policies = append(tbl.Policies, descpb.PolicyDescriptor{ + ID: op.Policy.PolicyID, + }) + return nil +} + +func (i *immediateVisitor) SetPolicyName(ctx context.Context, op scop.SetPolicyName) error { + policy, err := i.checkOutPolicy(ctx, op.TableID, op.PolicyID) + if err != nil { + return err + } + policy.Name = op.Name + return nil +} + +func (i *immediateVisitor) RemovePolicy(ctx context.Context, op scop.RemovePolicy) error { + tbl, err := i.checkOutTable(ctx, op.Policy.TableID) + if err != nil { + return err + } + var found bool + for idx := range tbl.Policies { + if tbl.Policies[idx].ID == op.Policy.PolicyID { + tbl.Policies = append(tbl.Policies[:idx], tbl.Policies[idx+1:]...) + found = true + break + } + } + if !found { + return errors.AssertionFailedf("failed to find policy with ID %d in table %q (%d)", + op.Policy.PolicyID, tbl.GetName(), tbl.GetID()) + } + return nil +} diff --git a/pkg/sql/schemachanger/scop/immediate_mutation.go b/pkg/sql/schemachanger/scop/immediate_mutation.go index bd5a58c63de0..ba5312d1958e 100644 --- a/pkg/sql/schemachanger/scop/immediate_mutation.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation.go @@ -588,6 +588,26 @@ type RemoveTrigger struct { Trigger scpb.Trigger } +// AddPolicy adds a policy to a table. +type AddPolicy struct { + immediateMutationOp + Policy scpb.Policy +} + +// RemovePolicy removes a policy from a table. +type RemovePolicy struct { + immediateMutationOp + Policy scpb.Policy +} + +// SetPolicyName sets the name of a policy. +type SetPolicyName struct { + immediateMutationOp + TableID descpb.ID + PolicyID descpb.PolicyID + Name string +} + // UpdateTableBackReferencesInTypes updates back references to a table // in the specified types. type UpdateTableBackReferencesInTypes struct { diff --git a/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go index 1aa0f3344ee3..3e2dc02586a5 100644 --- a/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go @@ -91,6 +91,9 @@ type ImmediateMutationVisitor interface { SetTriggerFunctionCall(context.Context, SetTriggerFunctionCall) error SetTriggerForwardReferences(context.Context, SetTriggerForwardReferences) error RemoveTrigger(context.Context, RemoveTrigger) error + AddPolicy(context.Context, AddPolicy) error + RemovePolicy(context.Context, RemovePolicy) error + SetPolicyName(context.Context, SetPolicyName) error UpdateTableBackReferencesInTypes(context.Context, UpdateTableBackReferencesInTypes) error UpdateTypeBackReferencesInTypes(context.Context, UpdateTypeBackReferencesInTypes) error RemoveBackReferenceInTypes(context.Context, RemoveBackReferenceInTypes) error @@ -526,6 +529,21 @@ func (op RemoveTrigger) Visit(ctx context.Context, v ImmediateMutationVisitor) e return v.RemoveTrigger(ctx, op) } +// Visit is part of the ImmediateMutationOp interface. +func (op AddPolicy) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.AddPolicy(ctx, op) +} + +// Visit is part of the ImmediateMutationOp interface. +func (op RemovePolicy) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.RemovePolicy(ctx, op) +} + +// Visit is part of the ImmediateMutationOp interface. +func (op SetPolicyName) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.SetPolicyName(ctx, op) +} + // Visit is part of the ImmediateMutationOp interface. func (op UpdateTableBackReferencesInTypes) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpdateTableBackReferencesInTypes(ctx, op) diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index acee26dddfc6..10f06ce9efd0 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -87,6 +87,7 @@ message ElementProto { LDRJobIDs ldr_job_ids = 134 [(gogoproto.customname) = "LDRJobIDs", (gogoproto.moretags) = "parent:\"Table\""]; PartitionZoneConfig partition_zone_config = 135 [(gogoproto.moretags) = "parent:\"TablePartitioning\""]; Trigger trigger = 136 [(gogoproto.moretags) = "parent:\"Table, View\""]; + Policy policy = 137 [(gogoproto.moretags) = "parent:\"Table\""]; // Multi-region elements. TableLocalityGlobal table_locality_global = 110 [(gogoproto.moretags) = "parent:\"Table\""]; @@ -166,7 +167,10 @@ message ElementProto { TriggerFunctionCall trigger_function_call = 206 [(gogoproto.moretags) = "parent:\"Trigger\""]; TriggerDeps trigger_deps = 207 [(gogoproto.moretags) = "parent:\"Trigger\""]; - // Next element group start id: 240 + // Policy elements. + PolicyName policy_name = 240 [(gogoproto.moretags) = "parent:\"Policy\""]; + + // Next element group start id: 250 } } @@ -357,6 +361,22 @@ message SchemaChild { uint32 schema_id = 2 [(gogoproto.customname) = "SchemaID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; } +// Policy represents a row-level security policy for a table. +// Note: This struct only stores the immutable parts of a policy. Any mutable +// components should be stored in separate elements so that an ALTER can update +// the policy by add/drop new elements. +message Policy { + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; + uint32 policy_id = 2 [(gogoproto.customname) = "PolicyID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.PolicyID"]; +} + +// PolicyName is the name assigned to a specific policy, based on its ID. +message PolicyName { + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; + uint32 policy_id = 2 [(gogoproto.customname) = "PolicyID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.PolicyID"]; + string name = 3; +} + message Sequence { uint32 sequence_id = 1 [(gogoproto.customname) = "SequenceID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; int64 restart_with = 2; diff --git a/pkg/sql/schemachanger/scpb/elements_generated.go b/pkg/sql/schemachanger/scpb/elements_generated.go index 08150f49ede6..550abfc87b72 100644 --- a/pkg/sql/schemachanger/scpb/elements_generated.go +++ b/pkg/sql/schemachanger/scpb/elements_generated.go @@ -1675,6 +1675,80 @@ func (c *ElementCollection[E]) FilterPartitionZoneConfig() *ElementCollection[*P return (*ElementCollection[*PartitionZoneConfig])(ret) } +func (e Policy) element() {} + +// Element implements ElementGetter. +func (e * ElementProto_Policy) Element() Element { + return e.Policy +} + +// ForEachPolicy iterates over elements of type Policy. +// Deprecated +func ForEachPolicy( + c *ElementCollection[Element], fn func(current Status, target TargetStatus, e *Policy), +) { + c.FilterPolicy().ForEach(fn) +} + +// FindPolicy finds the first element of type Policy. +// Deprecated +func FindPolicy( + c *ElementCollection[Element], +) (current Status, target TargetStatus, element *Policy) { + if tc := c.FilterPolicy(); !tc.IsEmpty() { + var e Element + current, target, e = tc.Get(0) + element = e.(*Policy) + } + return current, target, element +} + +// PolicyElements filters elements of type Policy. +func (c *ElementCollection[E]) FilterPolicy() *ElementCollection[*Policy] { + ret := c.genericFilter(func(_ Status, _ TargetStatus, e Element) bool { + _, ok := e.(*Policy) + return ok + }) + return (*ElementCollection[*Policy])(ret) +} + +func (e PolicyName) element() {} + +// Element implements ElementGetter. +func (e * ElementProto_PolicyName) Element() Element { + return e.PolicyName +} + +// ForEachPolicyName iterates over elements of type PolicyName. +// Deprecated +func ForEachPolicyName( + c *ElementCollection[Element], fn func(current Status, target TargetStatus, e *PolicyName), +) { + c.FilterPolicyName().ForEach(fn) +} + +// FindPolicyName finds the first element of type PolicyName. +// Deprecated +func FindPolicyName( + c *ElementCollection[Element], +) (current Status, target TargetStatus, element *PolicyName) { + if tc := c.FilterPolicyName(); !tc.IsEmpty() { + var e Element + current, target, e = tc.Get(0) + element = e.(*PolicyName) + } + return current, target, element +} + +// PolicyNameElements filters elements of type PolicyName. +func (c *ElementCollection[E]) FilterPolicyName() *ElementCollection[*PolicyName] { + ret := c.genericFilter(func(_ Status, _ TargetStatus, e Element) bool { + _, ok := e.(*PolicyName) + return ok + }) + return (*ElementCollection[*PolicyName])(ret) +} + func (e PrimaryIndex) element() {} // Element implements ElementGetter. @@ -3102,6 +3176,10 @@ func (e* ElementProto) SetElement(element Element) { e.ElementOneOf = &ElementProto_Owner{ Owner: t} case *PartitionZoneConfig: e.ElementOneOf = &ElementProto_PartitionZoneConfig{ PartitionZoneConfig: t} + case *Policy: + e.ElementOneOf = &ElementProto_Policy{ Policy: t} + case *PolicyName: + e.ElementOneOf = &ElementProto_PolicyName{ PolicyName: t} case *PrimaryIndex: e.ElementOneOf = &ElementProto_PrimaryIndex{ PrimaryIndex: t} case *RowLevelTTL: @@ -3225,6 +3303,8 @@ func GetElementOneOfProtos() []interface{} { ((*ElementProto_Namespace)(nil)), ((*ElementProto_Owner)(nil)), ((*ElementProto_PartitionZoneConfig)(nil)), + ((*ElementProto_Policy)(nil)), + ((*ElementProto_PolicyName)(nil)), ((*ElementProto_PrimaryIndex)(nil)), ((*ElementProto_RowLevelTTL)(nil)), ((*ElementProto_Schema)(nil)), @@ -3313,6 +3393,8 @@ func GetElementTypes() []interface{} { ((*Namespace)(nil)), ((*Owner)(nil)), ((*PartitionZoneConfig)(nil)), + ((*Policy)(nil)), + ((*PolicyName)(nil)), ((*PrimaryIndex)(nil)), ((*RowLevelTTL)(nil)), ((*Schema)(nil)), diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml index 090fa61e3042..7b0cb398782c 100644 --- a/pkg/sql/schemachanger/scpb/uml/table.puml +++ b/pkg/sql/schemachanger/scpb/uml/table.puml @@ -298,6 +298,17 @@ PartitionZoneConfig : []SubzoneSpans PartitionZoneConfig : SeqNum PartitionZoneConfig : OldIdxRef +object Policy + +Policy : TableID +Policy : PolicyID + +object PolicyName + +PolicyName : TableID +PolicyName : PolicyID +PolicyName : Name + object PrimaryIndex PrimaryIndex : Index @@ -576,6 +587,8 @@ Schema <|-- Owner AliasType <|-- Owner EnumType <|-- Owner TablePartitioning <|-- PartitionZoneConfig +Table <|-- Policy +Policy <|-- PolicyName Table <|-- PrimaryIndex View <|-- PrimaryIndex Table <|-- RowLevelTTL diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel index a14ca8cf48ef..86747623a8a5 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel @@ -50,6 +50,8 @@ go_library( "opgen_namespace.go", "opgen_owner.go", "opgen_partition_zone_config.go", + "opgen_policy.go", + "opgen_policy_name.go", "opgen_primary_index.go", "opgen_row_level_ttl.go", "opgen_schema.go", diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy.go new file mode 100644 index 000000000000..acd7aa312480 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy.go @@ -0,0 +1,33 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package opgen + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" +) + +func init() { + opRegistry.register((*scpb.Policy)(nil), + toPublic( + scpb.Status_ABSENT, + to(scpb.Status_PUBLIC, + emit(func(this *scpb.Policy) *scop.AddPolicy { + return &scop.AddPolicy{Policy: *protoutil.Clone(this).(*scpb.Policy)} + }), + ), + ), + toAbsent( + scpb.Status_PUBLIC, + to(scpb.Status_ABSENT, + emit(func(this *scpb.Policy) *scop.RemovePolicy { + return &scop.RemovePolicy{Policy: *protoutil.Clone(this).(*scpb.Policy)} + }), + ), + ), + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy_name.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy_name.go new file mode 100644 index 000000000000..c0c711eeba6c --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_policy_name.go @@ -0,0 +1,41 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package opgen + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" +) + +func init() { + opRegistry.register((*scpb.PolicyName)(nil), + toPublic( + scpb.Status_ABSENT, + to(scpb.Status_PUBLIC, + emit(func(this *scpb.PolicyName) *scop.SetPolicyName { + return &scop.SetPolicyName{ + TableID: this.TableID, + PolicyID: this.PolicyID, + Name: this.Name, + } + }), + ), + ), + toAbsent( + scpb.Status_PUBLIC, + to(scpb.Status_ABSENT, + emit(func(this *scpb.PolicyName) *scop.SetPolicyName { + return &scop.SetPolicyName{ + TableID: this.TableID, + PolicyID: this.PolicyID, + Name: tabledesc.PolicyNamePlaceholder(this.PolicyID), + } + }), + ), + ), + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel index 9f14bfb848ba..8de96fa12023 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel @@ -12,11 +12,13 @@ go_library( "dep_alter_column_type.go", "dep_create.go", "dep_create_function.go", + "dep_create_policy.go", "dep_drop_column.go", "dep_drop_constraint.go", "dep_drop_index.go", "dep_drop_index_and_column.go", "dep_drop_object.go", + "dep_drop_policy.go", "dep_drop_trigger.go", "dep_garbage_collection.go", "dep_swap_index.go", diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_create_policy.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_create_policy.go new file mode 100644 index 000000000000..c5d1be20df19 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_create_policy.go @@ -0,0 +1,29 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package current + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +func init() { + registerDepRule( + "dependents added after policy", + scgraph.SameStagePrecedence, + "policy", "dependent", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.Policy)(nil)), + to.TypeFilter(rulesVersionKey, isPolicyDependent), + JoinOnPolicyID(from, to, "table-id", "policy-id"), + StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), + } + }, + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_policy.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_policy.go new file mode 100644 index 000000000000..52aece347e45 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_policy.go @@ -0,0 +1,29 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package current + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + . "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" +) + +func init() { + registerDepRuleForDrop( + "dependents removed before policy", + scgraph.SameStagePrecedence, + "dependent", "policy", + scpb.Status_ABSENT, scpb.Status_ABSENT, + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.TypeFilter(rulesVersionKey, isPolicyDependent), + to.Type((*scpb.Policy)(nil)), + JoinOnPolicyID(from, to, "table-id", "policy-id"), + } + }, + ) +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go index 6907e78d3133..5b14e4a285bb 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go @@ -316,6 +316,14 @@ func isTriggerDependent(e scpb.Element) bool { return false } +func isPolicyDependent(e scpb.Element) bool { + switch e.(type) { + case *scpb.PolicyName: + return true + } + return false +} + func isData(e scpb.Element) bool { switch e.(type) { case *scpb.DatabaseData: diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index ad843541592c..59e4426c162d 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -2992,12 +2992,25 @@ deprules - $index-Node[CurrentStatus] = TRANSIENT_VALIDATED - joinTargetNode($view, $view-Target, $view-Node) - joinTargetNode($index, $index-Target, $index-Node) +- name: dependents added after policy + from: policy-Node + kind: SameStagePrecedence + to: dependent-Node + query: + - $policy[Type] = '*scpb.Policy' + - $dependent[Type] = '*scpb.PolicyName' + - joinOnPolicyID($policy, $dependent, $table-id, $policy-id) + - ToPublicOrTransient($policy-Target, $dependent-Target) + - $policy-Node[CurrentStatus] = PUBLIC + - $dependent-Node[CurrentStatus] = PUBLIC + - joinTargetNode($policy, $policy-Target, $policy-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - name: dependents exist before descriptor becomes public from: dependent-Node kind: Precedence to: relation-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $relation, $relation-id) - ToPublicOrTransient($dependent-Target, $relation-Target) @@ -3167,6 +3180,60 @@ deprules - $index-Node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - joinTargetNode($index, $index-Target, $index-Node) +- name: dependents removed before policy + from: dependent-Node + kind: SameStagePrecedence + to: policy-Node + query: + - $dependent[Type] = '*scpb.PolicyName' + - $policy[Type] = '*scpb.Policy' + - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) + - toAbsent($dependent-Target, $policy-Target) + - $dependent-Node[CurrentStatus] = ABSENT + - $policy-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($policy, $policy-Target, $policy-Node) +- name: dependents removed before policy + from: dependent-Node + kind: SameStagePrecedence + to: policy-Node + query: + - $dependent[Type] = '*scpb.PolicyName' + - $policy[Type] = '*scpb.Policy' + - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) + - transient($dependent-Target, $policy-Target) + - $dependent-Node[CurrentStatus] = TRANSIENT_ABSENT + - $policy-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($policy, $policy-Target, $policy-Node) +- name: dependents removed before policy + from: dependent-Node + kind: SameStagePrecedence + to: policy-Node + query: + - $dependent[Type] = '*scpb.PolicyName' + - $policy[Type] = '*scpb.Policy' + - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) + - $dependent-Target[TargetStatus] = TRANSIENT_ABSENT + - $dependent-Node[CurrentStatus] = TRANSIENT_ABSENT + - $policy-Target[TargetStatus] = ABSENT + - $policy-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($policy, $policy-Target, $policy-Node) +- name: dependents removed before policy + from: dependent-Node + kind: SameStagePrecedence + to: policy-Node + query: + - $dependent[Type] = '*scpb.PolicyName' + - $policy[Type] = '*scpb.Policy' + - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) + - $dependent-Target[TargetStatus] = ABSENT + - $dependent-Node[CurrentStatus] = ABSENT + - $policy-Target[TargetStatus] = TRANSIENT_ABSENT + - $policy-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($policy, $policy-Target, $policy-Node) - name: dependents removed right before simple constraint from: dependents-Node kind: SameStagePrecedence @@ -3241,7 +3308,7 @@ deprules to: referencing-via-attr-Node query: - $referenced-descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -3297,7 +3364,7 @@ deprules to: dependent-Node query: - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -3336,7 +3403,7 @@ deprules to: dependent-Node query: - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($relation, $dependent, $relation-id) - ToPublicOrTransient($relation-Target, $dependent-Target) - $relation-Node[CurrentStatus] = DESCRIPTOR_ADDED @@ -3817,7 +3884,7 @@ deprules kind: Precedence to: descriptor-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $descriptor, $desc-id) - toAbsent($dependent-Target, $descriptor-Target) @@ -7581,12 +7648,25 @@ deprules - $index-Node[CurrentStatus] = TRANSIENT_VALIDATED - joinTargetNode($view, $view-Target, $view-Node) - joinTargetNode($index, $index-Target, $index-Node) +- name: dependents added after policy + from: policy-Node + kind: SameStagePrecedence + to: dependent-Node + query: + - $policy[Type] = '*scpb.Policy' + - $dependent[Type] = '*scpb.PolicyName' + - joinOnPolicyID($policy, $dependent, $table-id, $policy-id) + - ToPublicOrTransient($policy-Target, $dependent-Target) + - $policy-Node[CurrentStatus] = PUBLIC + - $dependent-Node[CurrentStatus] = PUBLIC + - joinTargetNode($policy, $policy-Target, $policy-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - name: dependents exist before descriptor becomes public from: dependent-Node kind: Precedence to: relation-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $relation, $relation-id) - ToPublicOrTransient($dependent-Target, $relation-Target) @@ -7756,6 +7836,60 @@ deprules - $index-Node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - joinTargetNode($index, $index-Target, $index-Node) +- name: dependents removed before policy + from: dependent-Node + kind: SameStagePrecedence + to: policy-Node + query: + - $dependent[Type] = '*scpb.PolicyName' + - $policy[Type] = '*scpb.Policy' + - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) + - toAbsent($dependent-Target, $policy-Target) + - $dependent-Node[CurrentStatus] = ABSENT + - $policy-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($policy, $policy-Target, $policy-Node) +- name: dependents removed before policy + from: dependent-Node + kind: SameStagePrecedence + to: policy-Node + query: + - $dependent[Type] = '*scpb.PolicyName' + - $policy[Type] = '*scpb.Policy' + - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) + - transient($dependent-Target, $policy-Target) + - $dependent-Node[CurrentStatus] = TRANSIENT_ABSENT + - $policy-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($policy, $policy-Target, $policy-Node) +- name: dependents removed before policy + from: dependent-Node + kind: SameStagePrecedence + to: policy-Node + query: + - $dependent[Type] = '*scpb.PolicyName' + - $policy[Type] = '*scpb.Policy' + - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) + - $dependent-Target[TargetStatus] = TRANSIENT_ABSENT + - $dependent-Node[CurrentStatus] = TRANSIENT_ABSENT + - $policy-Target[TargetStatus] = ABSENT + - $policy-Node[CurrentStatus] = ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($policy, $policy-Target, $policy-Node) +- name: dependents removed before policy + from: dependent-Node + kind: SameStagePrecedence + to: policy-Node + query: + - $dependent[Type] = '*scpb.PolicyName' + - $policy[Type] = '*scpb.Policy' + - joinOnPolicyID($dependent, $policy, $table-id, $policy-id) + - $dependent-Target[TargetStatus] = ABSENT + - $dependent-Node[CurrentStatus] = ABSENT + - $policy-Target[TargetStatus] = TRANSIENT_ABSENT + - $policy-Node[CurrentStatus] = TRANSIENT_ABSENT + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) + - joinTargetNode($policy, $policy-Target, $policy-Node) - name: dependents removed right before simple constraint from: dependents-Node kind: SameStagePrecedence @@ -7830,7 +7964,7 @@ deprules to: referencing-via-attr-Node query: - $referenced-descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $referencing-via-attr[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.RowLevelTTL', '*scpb.SchemaComment', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -7886,7 +8020,7 @@ deprules to: dependent-Node query: - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraintUnvalidated', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -7925,7 +8059,7 @@ deprules to: dependent-Node query: - $relation[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseData', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexData', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableData', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - joinOnDescID($relation, $dependent, $relation-id) - ToPublicOrTransient($relation-Target, $dependent-Target) - $relation-Node[CurrentStatus] = DESCRIPTOR_ADDED @@ -8406,7 +8540,7 @@ deprules kind: Precedence to: descriptor-Node query: - - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] + - $dependent[Type] IN ['*scpb.CheckConstraint', '*scpb.CheckConstraintUnvalidated', '*scpb.Column', '*scpb.ColumnComment', '*scpb.ColumnComputeExpression', '*scpb.ColumnDefaultExpression', '*scpb.ColumnFamily', '*scpb.ColumnName', '*scpb.ColumnNotNull', '*scpb.ColumnOnUpdateExpression', '*scpb.ColumnType', '*scpb.CompositeTypeAttrName', '*scpb.CompositeTypeAttrType', '*scpb.ConstraintComment', '*scpb.ConstraintWithoutIndexName', '*scpb.DatabaseComment', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseZoneConfig', '*scpb.EnumTypeValue', '*scpb.ForeignKeyConstraint', '*scpb.ForeignKeyConstraintUnvalidated', '*scpb.FunctionBody', '*scpb.FunctionLeakProof', '*scpb.FunctionName', '*scpb.FunctionNullInputBehavior', '*scpb.FunctionSecurity', '*scpb.FunctionVolatility', '*scpb.IndexColumn', '*scpb.IndexComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.IndexZoneConfig', '*scpb.LDRJobIDs', '*scpb.NamedRangeZoneConfig', '*scpb.Namespace', '*scpb.Owner', '*scpb.PartitionZoneConfig', '*scpb.Policy', '*scpb.PolicyName', '*scpb.PrimaryIndex', '*scpb.RowLevelTTL', '*scpb.SchemaChild', '*scpb.SchemaComment', '*scpb.SchemaParent', '*scpb.SecondaryIndex', '*scpb.SecondaryIndexPartial', '*scpb.SequenceOption', '*scpb.SequenceOwner', '*scpb.TableComment', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.TableLocalitySecondaryRegion', '*scpb.TablePartitioning', '*scpb.TableSchemaLocked', '*scpb.TableZoneConfig', '*scpb.TemporaryIndex', '*scpb.Trigger', '*scpb.TriggerDeps', '*scpb.TriggerEnabled', '*scpb.TriggerEvents', '*scpb.TriggerFunctionCall', '*scpb.TriggerName', '*scpb.TriggerTiming', '*scpb.TriggerTransition', '*scpb.TriggerWhen', '*scpb.TypeComment', '*scpb.UniqueWithoutIndexConstraint', '*scpb.UniqueWithoutIndexConstraintUnvalidated', '*scpb.UserPrivileges'] - $descriptor[Type] IN ['*scpb.AliasType', '*scpb.CompositeType', '*scpb.Database', '*scpb.EnumType', '*scpb.Function', '*scpb.Schema', '*scpb.Sequence', '*scpb.Table', '*scpb.View'] - joinOnDescID($dependent, $descriptor, $desc-id) - toAbsent($dependent-Target, $descriptor-Target) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go index 187a3f35faab..b742502cc29c 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go @@ -129,6 +129,11 @@ func JoinOnTriggerID(a, b NodeVars, relationIDVar, triggerID rel.Var) rel.Clause return joinOnTriggerIDUntyped(a.El, b.El, relationIDVar, triggerID) } +// JoinOnPolicyID joins elements on policy ID. +func JoinOnPolicyID(a, b NodeVars, relationIDVar, policyID rel.Var) rel.Clause { + return joinOnPolicyIDUntyped(a.El, b.El, relationIDVar, policyID) +} + // ColumnInIndex requires that a column exists within an index. func ColumnInIndex( indexColumn, index NodeVars, relationIDVar, columnIDVar, indexIDVar rel.Var, @@ -329,6 +334,16 @@ var ( } }, ) + joinOnPolicyIDUntyped = screl.Schema.Def4( + "joinOnPolicyID", "a", "b", "desc-id", "policy-id", func( + a, b, descID, policyID rel.Var, + ) rel.Clauses { + return rel.Clauses{ + JoinOnDescIDUntyped(a, b, descID), + policyID.Entities(screl.PolicyID, a, b), + } + }, + ) columnInIndexUntyped = screl.Schema.Def5( "ColumnInIndex", diff --git a/pkg/sql/schemachanger/screl/attr.go b/pkg/sql/schemachanger/screl/attr.go index 4efe6f227f28..d64894d43343 100644 --- a/pkg/sql/schemachanger/screl/attr.go +++ b/pkg/sql/schemachanger/screl/attr.go @@ -108,6 +108,9 @@ const ( // Usage is an attribute for column compute expression to identify why it's // being added. Usage + // PolicyID is an attribute for row-level security policies to uniquely + // identify a policy within a table. + PolicyID // AttrMax is the largest possible Attr value. // Note: add any new enum values before TargetStatus, leave these at the end. @@ -360,6 +363,16 @@ var elementSchemaOptions = []rel.SchemaOption{ rel.EntityAttr(DescID, "TableID"), rel.EntityAttr(TriggerID, "TriggerID"), ), + // Policy elements + rel.EntityMapping(t((*scpb.Policy)(nil)), + rel.EntityAttr(DescID, "TableID"), + rel.EntityAttr(PolicyID, "PolicyID"), + ), + rel.EntityMapping(t((*scpb.PolicyName)(nil)), + rel.EntityAttr(DescID, "TableID"), + rel.EntityAttr(PolicyID, "PolicyID"), + rel.EntityAttr(Name, "Name"), + ), // Common elements. rel.EntityMapping(t((*scpb.Namespace)(nil)), rel.EntityAttr(DescID, "DescriptorID"), diff --git a/pkg/sql/schemachanger/screl/attr_string.go b/pkg/sql/schemachanger/screl/attr_string.go index 8e35d5f067e1..276592258989 100644 --- a/pkg/sql/schemachanger/screl/attr_string.go +++ b/pkg/sql/schemachanger/screl/attr_string.go @@ -38,7 +38,8 @@ func _() { _ = x[TypeName-23] _ = x[PartitionName-24] _ = x[Usage-25] - _ = x[AttrMax-25] + _ = x[PolicyID-26] + _ = x[AttrMax-26] } func (i Attr) String() string { @@ -93,6 +94,8 @@ func (i Attr) String() string { return "PartitionName" case Usage: return "Usage" + case PolicyID: + return "PolicyID" default: return "Attr(" + strconv.FormatInt(int64(i), 10) + ")" } diff --git a/pkg/sql/schemachanger/screl/scalars.go b/pkg/sql/schemachanger/screl/scalars.go index d6d05f74f84f..eb7686e0024e 100644 --- a/pkg/sql/schemachanger/screl/scalars.go +++ b/pkg/sql/schemachanger/screl/scalars.go @@ -130,7 +130,7 @@ func VersionSupportsElementUse(el scpb.Element, version clusterversion.ClusterVe *scpb.TriggerWhen, *scpb.TriggerFunctionCall, *scpb.TriggerDeps: // These elements need v24.3 so they can be used without checking any version gates. return true - case *scpb.NamedRangeZoneConfig: + case *scpb.NamedRangeZoneConfig, *scpb.Policy, *scpb.PolicyName: return version.IsActive(clusterversion.V25_1) default: panic(errors.AssertionFailedf("unknown element %T", el)) diff --git a/pkg/sql/schemachanger/sctest/end_to_end.go b/pkg/sql/schemachanger/sctest/end_to_end.go index 58074ac446ee..d30e3ed150b1 100644 --- a/pkg/sql/schemachanger/sctest/end_to_end.go +++ b/pkg/sql/schemachanger/sctest/end_to_end.go @@ -140,6 +140,7 @@ func EndToEndSideEffects(t *testing.T, relTestCaseDir string, factory TestServer sd.TempTablesEnabled = true sd.ApplicationName = "" sd.EnableUniqueWithoutIndexConstraints = true // this allows `ADD UNIQUE WITHOUT INDEX` in the testing suite. + sd.RowLevelSecurityEnabled = true sd.SerialNormalizationMode = localData.SerialNormalizationMode })), sctestdeps.WithTestingKnobs(&scexec.TestingKnobs{ diff --git a/pkg/sql/schemachanger/sctest/framework.go b/pkg/sql/schemachanger/sctest/framework.go index 64ae471393c9..9bfe37db17a8 100644 --- a/pkg/sql/schemachanger/sctest/framework.go +++ b/pkg/sql/schemachanger/sctest/framework.go @@ -872,6 +872,9 @@ func executeSchemaChangeTxn( _, err = conn.ExecContext( ctx, "SET experimental_enable_temp_tables=true", ) + _, err = conn.ExecContext( + ctx, "SET enable_row_level_security=true", + ) if err != nil { return err } diff --git a/pkg/sql/schemachanger/sctest_generated_test.go b/pkg/sql/schemachanger/sctest_generated_test.go index 8957b9368ff6..7fad56fc7528 100644 --- a/pkg/sql/schemachanger/sctest_generated_test.go +++ b/pkg/sql/schemachanger/sctest_generated_test.go @@ -435,6 +435,13 @@ func TestEndToEndSideEffects_create_index_create_schema_separate_statements(t *t sctest.EndToEndSideEffects(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestEndToEndSideEffects_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.EndToEndSideEffects(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestEndToEndSideEffects_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -603,6 +610,13 @@ func TestEndToEndSideEffects_drop_multiple_columns_separate_statements(t *testin sctest.EndToEndSideEffects(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestEndToEndSideEffects_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.EndToEndSideEffects(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestEndToEndSideEffects_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1044,6 +1058,13 @@ func TestExecuteWithDMLInjection_create_index_create_schema_separate_statements( sctest.ExecuteWithDMLInjection(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestExecuteWithDMLInjection_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.ExecuteWithDMLInjection(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestExecuteWithDMLInjection_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1212,6 +1233,13 @@ func TestExecuteWithDMLInjection_drop_multiple_columns_separate_statements(t *te sctest.ExecuteWithDMLInjection(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestExecuteWithDMLInjection_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.ExecuteWithDMLInjection(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestExecuteWithDMLInjection_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1653,6 +1681,13 @@ func TestGenerateSchemaChangeCorpus_create_index_create_schema_separate_statemen sctest.GenerateSchemaChangeCorpus(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestGenerateSchemaChangeCorpus_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.GenerateSchemaChangeCorpus(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestGenerateSchemaChangeCorpus_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -1821,6 +1856,13 @@ func TestGenerateSchemaChangeCorpus_drop_multiple_columns_separate_statements(t sctest.GenerateSchemaChangeCorpus(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestGenerateSchemaChangeCorpus_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.GenerateSchemaChangeCorpus(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestGenerateSchemaChangeCorpus_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -2262,6 +2304,13 @@ func TestPause_create_index_create_schema_separate_statements(t *testing.T) { sctest.Pause(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestPause_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.Pause(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestPause_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -2430,6 +2479,13 @@ func TestPause_drop_multiple_columns_separate_statements(t *testing.T) { sctest.Pause(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestPause_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.Pause(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestPause_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -2871,6 +2927,13 @@ func TestPauseMixedVersion_create_index_create_schema_separate_statements(t *tes sctest.PauseMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestPauseMixedVersion_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.PauseMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestPauseMixedVersion_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -3039,6 +3102,13 @@ func TestPauseMixedVersion_drop_multiple_columns_separate_statements(t *testing. sctest.PauseMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestPauseMixedVersion_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.PauseMixedVersion(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestPauseMixedVersion_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -3480,6 +3550,13 @@ func TestRollback_create_index_create_schema_separate_statements(t *testing.T) { sctest.Rollback(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestRollback_create_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/create_policy" + sctest.Rollback(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestRollback_create_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -3648,6 +3725,13 @@ func TestRollback_drop_multiple_columns_separate_statements(t *testing.T) { sctest.Rollback(t, path, sctest.SingleNodeTestClusterFactory{}) } +func TestRollback_drop_policy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const path = "pkg/sql/schemachanger/testdata/end_to_end/drop_policy" + sctest.Rollback(t, path, sctest.SingleNodeTestClusterFactory{}) +} + func TestRollback_drop_schema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.definition b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.definition new file mode 100644 index 000000000000..fbf50f7b20df --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.definition @@ -0,0 +1,7 @@ +setup +CREATE TABLE t1 (tenant_id uuid, c1 int); +---- + +test +CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT TO PUBLIC USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); +---- diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain new file mode 100644 index 000000000000..7fd679bc32e9 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain @@ -0,0 +1,29 @@ +/* setup */ +CREATE TABLE t1 (tenant_id uuid, c1 int); + +/* test */ +EXPLAIN (DDL) CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT TO PUBLIC USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); +---- +Schema change plan for CREATE POLICY ‹"policy 1"› ON ‹t1› AS PERMISSIVE FOR SELECT TO public USING (‹tenant_id› = ‹'01538898-f55c-44db-a306-89078e2c430e'›); + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 2 elements transitioning toward PUBLIC + │ │ ├── ABSENT → PUBLIC Policy:{DescID: 104 (t1), PolicyID: 1} + │ │ └── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} + │ └── 2 Mutation operations + │ ├── AddPolicy {"Policy":{"PolicyID":1,"TableID":104}} + │ └── SetPolicyName {"Name":"policy 1","PolicyID":1,"TableID":104} + └── PreCommitPhase + ├── Stage 1 of 2 in PreCommitPhase + │ ├── 2 elements transitioning toward PUBLIC + │ │ ├── PUBLIC → ABSENT Policy:{DescID: 104 (t1), PolicyID: 1} + │ │ └── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} + │ └── 1 Mutation operation + │ └── UndoAllInTxnImmediateMutationOpSideEffects + └── Stage 2 of 2 in PreCommitPhase + ├── 2 elements transitioning toward PUBLIC + │ ├── ABSENT → PUBLIC Policy:{DescID: 104 (t1), PolicyID: 1} + │ └── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} + └── 2 Mutation operations + ├── AddPolicy {"Policy":{"PolicyID":1,"TableID":104}} + └── SetPolicyName {"Name":"policy 1","PolicyID":1,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain_shape b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain_shape new file mode 100644 index 000000000000..2a9e01cc36d8 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain_shape @@ -0,0 +1,8 @@ +/* setup */ +CREATE TABLE t1 (tenant_id uuid, c1 int); + +/* test */ +EXPLAIN (DDL, SHAPE) CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT TO PUBLIC USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); +---- +Schema change plan for CREATE POLICY ‹"policy 1"› ON ‹t1› AS PERMISSIVE FOR SELECT TO public USING (‹tenant_id› = ‹'01538898-f55c-44db-a306-89078e2c430e'›); + └── execute 1 system table mutations transaction diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects new file mode 100644 index 000000000000..7e74501f6fac --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects @@ -0,0 +1,55 @@ +/* setup */ +CREATE TABLE t1 (tenant_id uuid, c1 int); +---- +... ++object {100 101 t1} -> 104 + +/* test */ +CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR SELECT TO PUBLIC USING (tenant_id = '01538898-f55c-44db-a306-89078e2c430e'); +---- +begin transaction #1 +# begin StatementPhase +checking for feature: CREATE POLICY +increment telemetry for sql.schema.create_policy +## StatementPhase stage 1 of 1 with 2 MutationType ops +upsert descriptor #104 + ... + nextIndexId: 2 + nextMutationId: 1 + + nextPolicyId: 2 + parentId: 100 + + policies: + + - id: 1 + + name: policy 1 + primaryIndex: + constraintId: 1 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 2 with 1 MutationType op +undo all catalog changes within txn #1 +persist all catalog changes to storage +## PreCommitPhase stage 2 of 2 with 2 MutationType ops +upsert descriptor #104 + ... + nextIndexId: 2 + nextMutationId: 1 + + nextPolicyId: 2 + parentId: 100 + + policies: + + - id: 1 + + name: policy 1 + primaryIndex: + constraintId: 1 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +persist all catalog changes to storage +# end PreCommitPhase +commit transaction #1 diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.definition b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.definition new file mode 100644 index 000000000000..c4897f39491e --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.definition @@ -0,0 +1,10 @@ +setup +CREATE TABLE t1 (tenant_id uuid, c1 int); +SET enable_row_level_security = true; +CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR ALL TO PUBLIC USING (true); +CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO PUBLIC WITH CHECK (true); +---- + +test +DROP POLICY "policy 2" on t1; +---- diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain new file mode 100644 index 000000000000..62d9ecdf1557 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain @@ -0,0 +1,32 @@ +/* setup */ +CREATE TABLE t1 (tenant_id uuid, c1 int); +SET enable_row_level_security = true; +CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR ALL TO PUBLIC USING (true); +CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO PUBLIC WITH CHECK (true); + +/* test */ +EXPLAIN (DDL) DROP POLICY "policy 2" on t1; +---- +Schema change plan for DROP POLICY ‹"policy 2"› ON ‹t1›; + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT Policy:{DescID: 104 (t1), PolicyID: 2} + │ │ └── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} + │ └── 2 Mutation operations + │ ├── SetPolicyName {"Name":"crdb_internal_po...","PolicyID":2,"TableID":104} + │ └── RemovePolicy {"Policy":{"PolicyID":2,"TableID":104}} + └── PreCommitPhase + ├── Stage 1 of 2 in PreCommitPhase + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── ABSENT → PUBLIC Policy:{DescID: 104 (t1), PolicyID: 2} + │ │ └── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} + │ └── 1 Mutation operation + │ └── UndoAllInTxnImmediateMutationOpSideEffects + └── Stage 2 of 2 in PreCommitPhase + ├── 2 elements transitioning toward ABSENT + │ ├── PUBLIC → ABSENT Policy:{DescID: 104 (t1), PolicyID: 2} + │ └── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} + └── 2 Mutation operations + ├── SetPolicyName {"Name":"crdb_internal_po...","PolicyID":2,"TableID":104} + └── RemovePolicy {"Policy":{"PolicyID":2,"TableID":104}} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain_shape b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain_shape new file mode 100644 index 000000000000..38b62dd01d6c --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain_shape @@ -0,0 +1,11 @@ +/* setup */ +CREATE TABLE t1 (tenant_id uuid, c1 int); +SET enable_row_level_security = true; +CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR ALL TO PUBLIC USING (true); +CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO PUBLIC WITH CHECK (true); + +/* test */ +EXPLAIN (DDL, SHAPE) DROP POLICY "policy 2" on t1; +---- +Schema change plan for DROP POLICY ‹"policy 2"› ON ‹t1›; + └── execute 1 system table mutations transaction diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects new file mode 100644 index 000000000000..3b49e01b77ca --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects @@ -0,0 +1,52 @@ +/* setup */ +CREATE TABLE t1 (tenant_id uuid, c1 int); +SET enable_row_level_security = true; +CREATE POLICY "policy 1" on t1 AS PERMISSIVE FOR ALL TO PUBLIC USING (true); +CREATE POLICY "policy 2" on t1 AS PERMISSIVE FOR INSERT TO PUBLIC WITH CHECK (true); +---- +... ++object {100 101 t1} -> 104 + +/* test */ +DROP POLICY "policy 2" on t1; +---- +begin transaction #1 +# begin StatementPhase +checking for feature: DROP POLICY +increment telemetry for sql.schema.drop_policy +## StatementPhase stage 1 of 1 with 2 MutationType ops +upsert descriptor #104 + ... + - id: 1 + name: policy 1 + - - id: 2 + - name: policy 2 + primaryIndex: + constraintId: 1 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "3" + + version: "4" +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 2 with 1 MutationType op +undo all catalog changes within txn #1 +persist all catalog changes to storage +## PreCommitPhase stage 2 of 2 with 2 MutationType ops +upsert descriptor #104 + ... + - id: 1 + name: policy 1 + - - id: 2 + - name: policy 2 + primaryIndex: + constraintId: 1 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "3" + + version: "4" +persist all catalog changes to storage +# end PreCommitPhase +commit transaction #1 diff --git a/pkg/sql/sem/catid/ids.go b/pkg/sql/sem/catid/ids.go index 3cefc6f82e18..f2769ef310ae 100644 --- a/pkg/sql/sem/catid/ids.go +++ b/pkg/sql/sem/catid/ids.go @@ -87,3 +87,9 @@ func (PGAttributeNum) SafeValue() {} // RoleID is a custom type for a role id. type RoleID uint32 + +// PolicyID is a custom type for TableDescriptor policy IDs. +type PolicyID uint32 + +// SafeValue implements the redact.SafeValue interface. +func (PolicyID) SafeValue() {} diff --git a/pkg/sql/sem/tree/alter_policy.go b/pkg/sql/sem/tree/alter_policy.go index 6414fd82851e..54c260bfcefd 100644 --- a/pkg/sql/sem/tree/alter_policy.go +++ b/pkg/sql/sem/tree/alter_policy.go @@ -10,7 +10,7 @@ var _ Statement = &AlterPolicy{} // AlterPolicy is a tree struct for the ALTER POLICY DDL statement type AlterPolicy struct { PolicyName Name - TableName TableName + TableName *UnresolvedObjectName NewPolicyName Name Roles RoleSpecList Exprs PolicyExpressions @@ -21,7 +21,7 @@ func (node *AlterPolicy) Format(ctx *FmtCtx) { ctx.WriteString("ALTER POLICY ") ctx.FormatNode(&node.PolicyName) ctx.WriteString(" ON ") - ctx.FormatNode(&node.TableName) + ctx.FormatNode(node.TableName) if node.NewPolicyName != "" { ctx.WriteString(" RENAME TO ") diff --git a/pkg/sql/sem/tree/create_policy.go b/pkg/sql/sem/tree/create_policy.go index 4f19d33123bd..5ad96a155ea5 100644 --- a/pkg/sql/sem/tree/create_policy.go +++ b/pkg/sql/sem/tree/create_policy.go @@ -79,7 +79,7 @@ func (node *PolicyExpressions) Format(ctx *FmtCtx) { // CreatePolicy is a tree struct for the CREATE POLICY DDL statement type CreatePolicy struct { PolicyName Name - TableName TableName + TableName *UnresolvedObjectName Type PolicyType Cmd PolicyCommand Roles RoleSpecList @@ -91,7 +91,7 @@ func (node *CreatePolicy) Format(ctx *FmtCtx) { ctx.WriteString("CREATE POLICY ") ctx.FormatNode(&node.PolicyName) ctx.WriteString(" ON ") - ctx.FormatNode(&node.TableName) + ctx.FormatNode(node.TableName) if node.Type != PolicyTypeDefault { ctx.WriteString(" AS ") ctx.WriteString(node.Type.String()) diff --git a/pkg/sql/sqlerrors/errors.go b/pkg/sql/sqlerrors/errors.go index 8e13448df011..e67e7b20c02d 100644 --- a/pkg/sql/sqlerrors/errors.go +++ b/pkg/sql/sqlerrors/errors.go @@ -449,6 +449,12 @@ func NewUndefinedTriggerError(triggerName, tableName string) error { "trigger %q for table %q does not exist", triggerName, tableName) } +// NewUndefinedPolicyError return an error that the policy doesn't exist +func NewUndefinedPolicyError(policyName, tableName string) error { + return pgerror.Newf(pgcode.UndefinedObject, + "policy %q for table %q does not exist", policyName, tableName) +} + // NewRangeUnavailableError creates an unavailable range error. func NewRangeUnavailableError(rangeID roachpb.RangeID, origErr error) error { return pgerror.Wrapf(origErr, pgcode.RangeUnavailable, "key range id:%d is unavailable", rangeID) diff --git a/pkg/testutils/lint/passes/redactcheck/redactcheck.go b/pkg/testutils/lint/passes/redactcheck/redactcheck.go index ee955a6e4bfc..e6166aa736a6 100644 --- a/pkg/testutils/lint/passes/redactcheck/redactcheck.go +++ b/pkg/testutils/lint/passes/redactcheck/redactcheck.go @@ -226,6 +226,7 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "IndexID": {}, "PGAttributeNum": {}, "TriggerID": {}, + "PolicyID": {}, }, "github.com/cockroachdb/cockroach/pkg/sql/sem/tree": { "IsolationLevel": {}, From 43d099ad2d84f710a52d38312e340dc42ccd7cae Mon Sep 17 00:00:00 2001 From: Miles Frankel Date: Thu, 9 Jan 2025 16:40:50 -0500 Subject: [PATCH 032/126] changefeedccl: fix flaky tests Fix flaky tests. Fixes: #136989 Fixed: #131718 Release note: None --- .../changefeedccl/alter_changefeed_test.go | 58 +++++++++++++++---- 1 file changed, 47 insertions(+), 11 deletions(-) diff --git a/pkg/ccl/changefeedccl/alter_changefeed_test.go b/pkg/ccl/changefeedccl/alter_changefeed_test.go index 3e33dfbf7567..86a0aa35cd57 100644 --- a/pkg/ccl/changefeedccl/alter_changefeed_test.go +++ b/pkg/ccl/changefeedccl/alter_changefeed_test.go @@ -246,13 +246,30 @@ func TestAlterChangefeedAddTargetFamily(t *testing.T) { testFeed := feed(t, f, `CREATE CHANGEFEED FOR foo FAMILY onlya`) defer closeFeed(t, testFeed) - sqlDB.Exec(t, `INSERT INTO foo VALUES(1, 'hello')`) + feed, ok := testFeed.(cdctest.EnterpriseTestFeed) + require.True(t, ok) + + tsr := sqlDB.QueryRow(t, `INSERT INTO foo VALUES(42, 'hello') RETURNING cluster_logical_timestamp()`) + var insertTsDecStr string + tsr.Scan(&insertTsDecStr) + insertTs := parseTimeToHLC(t, insertTsDecStr) assertPayloads(t, testFeed, []string{ - `foo.onlya: [1]->{"after": {"a": 1}}`, + `foo.onlya: [42]->{"after": {"a": 42}}`, }) - feed, ok := testFeed.(cdctest.EnterpriseTestFeed) - require.True(t, ok) + // Wait for the high water mark (aka resolved ts) to advance past the row we inserted's + // mvcc ts. Otherwise, we'd see [42] again due to a catch up scan, and it + // would muddy the waters. + testutils.SucceedsSoon(t, func() error { + registry := s.Server.JobRegistry().(*jobs.Registry) + job, err := registry.LoadJob(context.Background(), feed.JobID()) + require.NoError(t, err) + prog := job.Progress() + if p := prog.GetHighWater(); p != nil && !p.IsEmpty() && insertTs.Less(*p) { + return nil + } + return errors.New("waiting for highwater") + }) sqlDB.Exec(t, `PAUSE JOB $1`, feed.JobID()) waitForJobStatus(sqlDB, t, feed.JobID(), `paused`) @@ -261,12 +278,12 @@ func TestAlterChangefeedAddTargetFamily(t *testing.T) { sqlDB.Exec(t, fmt.Sprintf(`RESUME JOB %d`, feed.JobID())) waitForJobStatus(sqlDB, t, feed.JobID(), `running`) - - sqlDB.Exec(t, `INSERT INTO foo VALUES(2, 'goodbye')`) + sqlDB.Exec(t, `INSERT INTO foo VALUES(37, 'goodbye')`) assertPayloads(t, testFeed, []string{ - `foo.onlyb: [1]->{"after": {"b": "hello"}}`, - `foo.onlya: [2]->{"after": {"a": 2}}`, - `foo.onlyb: [2]->{"after": {"b": "goodbye"}}`, + // Note that we don't see foo.onlyb.[42] here, because we're not + // doing a catchup scan and we've already processed that tuple. + `foo.onlya: [37]->{"after": {"a": 37}}`, + `foo.onlyb: [37]->{"after": {"b": "goodbye"}}`, }) } @@ -287,7 +304,11 @@ func TestAlterChangefeedSwitchFamily(t *testing.T) { testFeed := feed(t, f, `CREATE CHANGEFEED FOR foo FAMILY onlya`) defer closeFeed(t, testFeed) - sqlDB.Exec(t, `INSERT INTO foo VALUES(1, 'hello')`) + tsr := sqlDB.QueryRow(t, `INSERT INTO foo VALUES(1, 'hello') RETURNING cluster_logical_timestamp()`) + var insertTsDecStr string + tsr.Scan(&insertTsDecStr) + insertTs := parseTimeToHLC(t, insertTsDecStr) + assertPayloads(t, testFeed, []string{ `foo.onlya: [1]->{"after": {"a": 1}}`, }) @@ -295,6 +316,20 @@ func TestAlterChangefeedSwitchFamily(t *testing.T) { feed, ok := testFeed.(cdctest.EnterpriseTestFeed) require.True(t, ok) + // Wait for the high water mark (aka resolved ts) to advance past the row we inserted's + // mvcc ts. Otherwise, we'd see [1] again due to a catch up scan, and it + // would muddy the waters. + testutils.SucceedsSoon(t, func() error { + registry := s.Server.JobRegistry().(*jobs.Registry) + job, err := registry.LoadJob(context.Background(), feed.JobID()) + require.NoError(t, err) + prog := job.Progress() + if p := prog.GetHighWater(); p != nil && !p.IsEmpty() && insertTs.Less(*p) { + return nil + } + return errors.New("waiting for highwater") + }) + sqlDB.Exec(t, `PAUSE JOB $1`, feed.JobID()) waitForJobStatus(sqlDB, t, feed.JobID(), `paused`) @@ -305,7 +340,8 @@ func TestAlterChangefeedSwitchFamily(t *testing.T) { sqlDB.Exec(t, `INSERT INTO foo VALUES(2, 'goodbye')`) assertPayloads(t, testFeed, []string{ - `foo.onlyb: [1]->{"after": {"b": "hello"}}`, + // Note that we don't see foo.onlyb.[1] here, because we're not + // doing a catchup scan and we've already processed that tuple. `foo.onlyb: [2]->{"after": {"b": "goodbye"}}`, }) } From e6401dc062a3e891a1fa95e843a8598ea4fcc4f5 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Wed, 8 Jan 2025 17:52:57 -0500 Subject: [PATCH 033/126] sql/parser: introduce more LDR options for bidi replication This patch introduces two options to CREATE LOGICALLY REPLICATED TABLES: - UNIDIRECTIONAL: to specify a unidectional ldr stream with a fast offline scan - BIDIRECTIONAL ON {uri}: which customers will use to set up a bidirectional stream with one command. The uri provided by the customer connects to the source of the reverse stream, aka the destination of the main stream. The PARENT option will be used internally to set up the reverse stream. Specifically, when the destination sets up the reverse stream, it will pass its jobID as the parent for the reverse stream, which the reverse stream will use to prevent creating duplicate reverse streams and to configure a stream that replicates from an offline source. Customers should not directly use it. Epic: none Release note: none --- docs/generated/sql/bnf/stmt_block.bnf | 6 ++ pkg/sql/parser/sql.y | 23 +++++++- .../testdata/create_logical_replication | 16 +++++ .../sem/tree/create_logical_replication.go | 59 ++++++++++++++++--- 4 files changed, 94 insertions(+), 10 deletions(-) diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 03fa95526eae..15cf61aa499f 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1087,6 +1087,7 @@ unreserved_keyword ::= | 'BATCH' | 'BEFORE' | 'BEGIN' + | 'BIDIRECTIONAL' | 'BINARY' | 'BUCKET_COUNT' | 'BUNDLE' @@ -1513,6 +1514,7 @@ unreserved_keyword ::= | 'TYPE' | 'TYPES' | 'THROTTLING' + | 'UNIDIRECTIONAL' | 'UNBOUNDED' | 'UNCOMMITTED' | 'UNKNOWN' @@ -3360,6 +3362,8 @@ logical_replication_create_table_options ::= 'MODE' '=' string_or_placeholder | 'DISCARD' '=' string_or_placeholder | 'LABEL' '=' string_or_placeholder + | 'UNIDIRECTIONAL' + | 'BIDIRECTIONAL' 'ON' string_or_placeholder common_table_expr ::= table_alias_name opt_col_def_list_no_types 'AS' materialize_clause '(' preparable_stmt ')' @@ -3774,6 +3778,7 @@ bare_label_keywords ::= | 'BEFORE' | 'BEGIN' | 'BETWEEN' + | 'BIDIRECTIONAL' | 'BIGINT' | 'BINARY' | 'BIT' @@ -4302,6 +4307,7 @@ bare_label_keywords ::= | 'TYPES' | 'UNBOUNDED' | 'UNCOMMITTED' + | 'UNIDIRECTIONAL' | 'UNIQUE' | 'UNKNOWN' | 'UNLISTEN' diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 808fb12d2550..5361931229d1 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -969,7 +969,7 @@ func (u *sqlSymUnion) doBlockOption() tree.DoBlockOption { %token ALL ALTER ALWAYS ANALYSE ANALYZE AND AND_AND ANY ANNOTATE_TYPE ARRAY AS ASC AS_JSON AT_AT %token ASENSITIVE ASYMMETRIC AT ATOMIC ATTRIBUTE AUTHORIZATION AUTOMATIC AVAILABILITY AVOID_FULL_SCAN -%token BACKUP BACKUPS BACKWARD BATCH BEFORE BEGIN BETWEEN BIGINT BIGSERIAL BINARY BIT +%token BACKUP BACKUPS BACKWARD BATCH BEFORE BEGIN BETWEEN BIDIRECTIONAL BIGINT BIGSERIAL BINARY BIT %token BUCKET_COUNT %token BOOLEAN BOTH BOX2D BUNDLE BY BYPASSRLS @@ -1066,7 +1066,7 @@ func (u *sqlSymUnion) doBlockOption() tree.DoBlockOption { %token TRUNCATE TRUSTED TYPE TYPES %token TRACING -%token UNBOUNDED UNCOMMITTED UNION UNIQUE UNKNOWN UNLISTEN UNLOGGED UNSAFE_RESTORE_INCOMPATIBLE_VERSION UNSPLIT +%token UNBOUNDED UNCOMMITTED UNIDIRECTIONAL UNION UNIQUE UNKNOWN UNLISTEN UNLOGGED UNSAFE_RESTORE_INCOMPATIBLE_VERSION UNSPLIT %token UPDATE UPDATES_CLUSTER_MONITORING_METRICS UPSERT UNSET UNTIL USE USER USERS USING UUID %token VALID VALIDATE VALUE VALUES VARBIT VARCHAR VARIADIC VECTOR VERIFY_BACKUP_TABLE_DATA VIEW VARIABLES VARYING VIEWACTIVITY VIEWACTIVITYREDACTED VIEWDEBUG @@ -4811,6 +4811,12 @@ logical_replication_options: { $$.val = &tree.LogicalReplicationOptions{MetricsLabel: $3.expr()} } +| PARENT '=' string_or_placeholder + /* SKIP DOC */ + { + $$.val = &tree.LogicalReplicationOptions{ParentID: $3.expr()} + } + logical_replication_create_table_options: MODE '=' string_or_placeholder @@ -4825,6 +4831,15 @@ logical_replication_create_table_options: { $$.val = &tree.LogicalReplicationOptions{MetricsLabel: $3.expr()} } +| UNIDIRECTIONAL + { + $$.val = &tree.LogicalReplicationOptions{Unidirectional: tree.MakeDBool(true)} + } +| BIDIRECTIONAL ON string_or_placeholder + { + $$.val = &tree.LogicalReplicationOptions{BidirectionalURI: $3.expr()} + } + // %Help: CREATE VIRTUAL CLUSTER - create a new virtual cluster // %Category: Experimental @@ -18009,6 +18024,7 @@ unreserved_keyword: | BATCH | BEFORE | BEGIN +| BIDIRECTIONAL | BINARY | BUCKET_COUNT | BUNDLE @@ -18435,6 +18451,7 @@ unreserved_keyword: | TYPE | TYPES | THROTTLING +| UNIDIRECTIONAL | UNBOUNDED | UNCOMMITTED | UNKNOWN @@ -18512,6 +18529,7 @@ bare_label_keywords: | BEFORE | BEGIN | BETWEEN +| BIDIRECTIONAL | BIGINT | BINARY | BIT @@ -19040,6 +19058,7 @@ bare_label_keywords: | TYPES | UNBOUNDED | UNCOMMITTED +| UNIDIRECTIONAL | UNIQUE | UNKNOWN | UNLISTEN diff --git a/pkg/sql/parser/testdata/create_logical_replication b/pkg/sql/parser/testdata/create_logical_replication index 9522a3661192..d1b73b676d08 100644 --- a/pkg/sql/parser/testdata/create_logical_replication +++ b/pkg/sql/parser/testdata/create_logical_replication @@ -22,6 +22,14 @@ CREATE LOGICALLY REPLICATED TABLE (foo) FROM TABLE (foo) ON ('uri') -- fully par CREATE LOGICALLY REPLICATED TABLE foo FROM TABLE foo ON '_' -- literals removed CREATE LOGICALLY REPLICATED TABLE _ FROM TABLE _ ON 'uri' -- identifiers removed +parse +CREATE LOGICALLY REPLICATED TABLE foo FROM TABLE foo ON 'uri' WITH BIDIRECTIONAL ON 'reverse'; +---- +CREATE LOGICALLY REPLICATED TABLE foo FROM TABLE foo ON 'uri' WITH OPTIONS (BIDIRECTIONAL ON 'reverse') -- normalized! +CREATE LOGICALLY REPLICATED TABLE (foo) FROM TABLE (foo) ON ('uri') WITH OPTIONS (BIDIRECTIONAL ON ('reverse')) -- fully parenthesized +CREATE LOGICALLY REPLICATED TABLE foo FROM TABLE foo ON '_' WITH OPTIONS (BIDIRECTIONAL ON '_') -- literals removed +CREATE LOGICALLY REPLICATED TABLE _ FROM TABLE _ ON 'uri' WITH OPTIONS (BIDIRECTIONAL ON 'reverse') -- identifiers removed + error CREATE LOGICALLY REPLICATED TABLE foo FROM TABLE foo ON 'uri' WITH CURSOR = '1536242855577149065.0000000000'; ---- @@ -71,6 +79,14 @@ CREATE LOGICAL REPLICATION STREAM FROM TABLE (foo) ON ('uri') INTO TABLE (foo) W CREATE LOGICAL REPLICATION STREAM FROM TABLE foo ON '_' INTO TABLE foo WITH OPTIONS (MODE = '_', FUNCTION a FOR TABLE b, FUNCTION c FOR TABLE d, LABEL = '_') -- literals removed CREATE LOGICAL REPLICATION STREAM FROM TABLE _ ON 'uri' INTO TABLE _ WITH OPTIONS (MODE = 'immediate', FUNCTION _ FOR TABLE _, FUNCTION _ FOR TABLE _, LABEL = 'foo') -- identifiers removed +parse +CREATE LOGICAL REPLICATION STREAM FROM TABLE foo ON 'uri' INTO TABLE foo WITH PARENT = '1036407336021721089'; +---- +CREATE LOGICAL REPLICATION STREAM FROM TABLE foo ON 'uri' INTO TABLE foo WITH OPTIONS (PARENT = '1036407336021721089') -- normalized! +CREATE LOGICAL REPLICATION STREAM FROM TABLE (foo) ON ('uri') INTO TABLE (foo) WITH OPTIONS (PARENT = ('1036407336021721089')) -- fully parenthesized +CREATE LOGICAL REPLICATION STREAM FROM TABLE foo ON '_' INTO TABLE foo WITH OPTIONS (PARENT = '_') -- literals removed +CREATE LOGICAL REPLICATION STREAM FROM TABLE _ ON 'uri' INTO TABLE _ WITH OPTIONS (PARENT = '1036407336021721089') -- identifiers removed + parse CREATE LOGICAL REPLICATION STREAM FROM TABLE foo ON 'uri' INTO TABLE foo WITH CURSOR = '1536242855577149065.0000000000', DEFAULT FUNCTION = 'lww', FUNCTION a FOR TABLE b, MODE = 'immediate', FUNCTION c FOR TABLE d ---- diff --git a/pkg/sql/sem/tree/create_logical_replication.go b/pkg/sql/sem/tree/create_logical_replication.go index 417bfd93905a..0950bf0b5b94 100644 --- a/pkg/sql/sem/tree/create_logical_replication.go +++ b/pkg/sql/sem/tree/create_logical_replication.go @@ -26,13 +26,16 @@ type LogicalReplicationResources struct { type LogicalReplicationOptions struct { // Mapping of table name to UDF name - UserFunctions map[UnresolvedName]RoutineName - Cursor Expr - MetricsLabel Expr - Mode Expr - DefaultFunction Expr - Discard Expr - SkipSchemaCheck *DBool + UserFunctions map[UnresolvedName]RoutineName + Cursor Expr + MetricsLabel Expr + Mode Expr + DefaultFunction Expr + Discard Expr + SkipSchemaCheck *DBool + Unidirectional *DBool + BidirectionalURI Expr + ParentID Expr } var _ Statement = &CreateLogicalReplicationStream{} @@ -149,6 +152,21 @@ func (lro *LogicalReplicationOptions) Format(ctx *FmtCtx) { ctx.FormatNode(lro.MetricsLabel) } + if lro.Unidirectional != nil { + maybeAddSep() + ctx.WriteString("UNIDIRECTIONAL") + } + if lro.BidirectionalURI != nil { + maybeAddSep() + ctx.WriteString("BIDIRECTIONAL ON ") + ctx.FormatNode(lro.BidirectionalURI) + } + if lro.ParentID != nil { + maybeAddSep() + ctx.WriteString("PARENT = ") + ctx.FormatNode(lro.ParentID) + } + } func (o *LogicalReplicationOptions) CombineWith(other *LogicalReplicationOptions) error { @@ -211,6 +229,28 @@ func (o *LogicalReplicationOptions) CombineWith(other *LogicalReplicationOptions o.MetricsLabel = other.MetricsLabel } + if o.Unidirectional != nil { + if other.Unidirectional != nil { + return errors.New("UNIDIRECTIONAL option specified multiple times") + } + } else { + o.Unidirectional = other.Unidirectional + } + if o.BidirectionalURI != nil { + if other.BidirectionalURI != nil { + return errors.New("BIDIRECTIONAL option specified multiple times") + } + } else { + o.BidirectionalURI = other.BidirectionalURI + } + if o.ParentID != nil { + if other.ParentID != nil { + return errors.New("PARENT option specified multiple times") + } + } else { + o.ParentID = other.ParentID + } + return nil } @@ -223,5 +263,8 @@ func (o LogicalReplicationOptions) IsDefault() bool { o.UserFunctions == nil && o.Discard == options.Discard && o.SkipSchemaCheck == options.SkipSchemaCheck && - o.MetricsLabel == options.MetricsLabel + o.MetricsLabel == options.MetricsLabel && + o.Unidirectional == options.Unidirectional && + o.BidirectionalURI == options.BidirectionalURI && + o.ParentID == options.ParentID } From 60496b1af87cdec4a323921da28f304031fdf207 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Wed, 8 Jan 2025 17:55:52 -0500 Subject: [PATCH 034/126] crosscluster/streamclient: add ExecStatement api This destination will use this api to set up a reverse stream from a cursor. Epic: none Release note: none --- pkg/crosscluster/streamclient/client.go | 1 + pkg/crosscluster/streamclient/client_test.go | 8 +++++++- pkg/crosscluster/streamclient/mock_stream_client.go | 6 ++++++ .../streamclient/partitioned_stream_client.go | 11 +++++++++++ .../streamclient/randclient/random_stream_client.go | 6 ++++++ 5 files changed, 31 insertions(+), 1 deletion(-) diff --git a/pkg/crosscluster/streamclient/client.go b/pkg/crosscluster/streamclient/client.go index 07205764f2a0..07d3bab9aebd 100644 --- a/pkg/crosscluster/streamclient/client.go +++ b/pkg/crosscluster/streamclient/client.go @@ -97,6 +97,7 @@ type Client interface { PlanLogicalReplication(ctx context.Context, req streampb.LogicalReplicationPlanRequest) (LogicalReplicationPlan, error) CreateForTables(ctx context.Context, req *streampb.ReplicationProducerRequest) (*streampb.ReplicationProducerSpec, error) + ExecStatement(ctx context.Context, cmd string, opname string, args ...interface{}) error // Close releases all the resources used by this client. Close(ctx context.Context) error diff --git a/pkg/crosscluster/streamclient/client_test.go b/pkg/crosscluster/streamclient/client_test.go index 8cbc723342ec..04358b579482 100644 --- a/pkg/crosscluster/streamclient/client_test.go +++ b/pkg/crosscluster/streamclient/client_test.go @@ -134,7 +134,13 @@ func (sc testStreamClient) Complete(_ context.Context, _ streampb.StreamID, _ bo return nil } -// PriorReplicationDetails implements the Client interface. +func (sc testStreamClient) ExecStatement( + _ context.Context, _ string, _ string, _ ...interface{}, +) error { + return nil +} + +// PriorReplicationDetails implements the streamclient.Client interface. func (sc testStreamClient) PriorReplicationDetails( _ context.Context, _ roachpb.TenantName, ) (string, string, hlc.Timestamp, error) { diff --git a/pkg/crosscluster/streamclient/mock_stream_client.go b/pkg/crosscluster/streamclient/mock_stream_client.go index 3ae613d8df13..0a77e05a45bc 100644 --- a/pkg/crosscluster/streamclient/mock_stream_client.go +++ b/pkg/crosscluster/streamclient/mock_stream_client.go @@ -146,6 +146,12 @@ func (p *MockStreamClient) CreateForTables( return nil, errors.AssertionFailedf("unimplemented") } +func (p *MockStreamClient) ExecStatement( + ctx context.Context, cmd string, opname string, args ...interface{}, +) error { + return errors.AssertionFailedf("unimplemented") +} + // ErrorStreamClient always returns an error when consuming a partition. type ErrorStreamClient struct{ MockStreamClient } diff --git a/pkg/crosscluster/streamclient/partitioned_stream_client.go b/pkg/crosscluster/streamclient/partitioned_stream_client.go index 48ee270e9644..a52c8b0bb707 100644 --- a/pkg/crosscluster/streamclient/partitioned_stream_client.go +++ b/pkg/crosscluster/streamclient/partitioned_stream_client.go @@ -392,6 +392,17 @@ func (p *partitionedStreamClient) CreateForTables( } return spec, nil } +func (p *partitionedStreamClient) ExecStatement( + ctx context.Context, cmd string, opname string, args ...interface{}, +) error { + ctx, sp := tracing.ChildSpan(ctx, opname) + defer sp.Finish() + + p.mu.Lock() + defer p.mu.Unlock() + _, err := p.mu.srcConn.Exec(ctx, cmd, args...) + return err +} // PriorReplicationDetails implements the Client interface. func (p *partitionedStreamClient) PriorReplicationDetails( diff --git a/pkg/crosscluster/streamclient/randclient/random_stream_client.go b/pkg/crosscluster/streamclient/randclient/random_stream_client.go index 615be000c2e7..e5ca30f90552 100644 --- a/pkg/crosscluster/streamclient/randclient/random_stream_client.go +++ b/pkg/crosscluster/streamclient/randclient/random_stream_client.go @@ -638,6 +638,12 @@ func (p *RandomStreamClient) PriorReplicationDetails( } +func (p *RandomStreamClient) ExecStatement( + ctx context.Context, cmd string, opname string, args ...interface{}, +) error { + return errors.AssertionFailedf("unimplemented") +} + type randomStreamSubscription struct { receiveFn func(ctx context.Context) error eventCh chan crosscluster.Event From 77ffc7464b5df5bf41d87f012c524f239c830226 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Fri, 10 Jan 2025 17:55:53 -0500 Subject: [PATCH 035/126] sql/externalcatalog: pass mutable flag to workaround resolver bug For mysterious resons, the resolver cannot fetch offline descriptors unless the mutable flag is passed. Epic: none Release note: none --- pkg/sql/catalog/externalcatalog/external_catalog.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pkg/sql/catalog/externalcatalog/external_catalog.go b/pkg/sql/catalog/externalcatalog/external_catalog.go index 11edf569d3a2..8e19b6d36800 100644 --- a/pkg/sql/catalog/externalcatalog/external_catalog.go +++ b/pkg/sql/catalog/externalcatalog/external_catalog.go @@ -49,9 +49,11 @@ func ExtractExternalCatalog( return externalpb.ExternalCatalog{}, err } lookupFlags := tree.ObjectLookupFlags{ - Required: true, - DesiredObjectKind: tree.TableObject, - IncludeOffline: includeOffline, + Required: true, + DesiredObjectKind: tree.TableObject, + IncludeOffline: includeOffline, + DesiredTableDescKind: tree.ResolveRequireTableDesc, + RequireMutable: includeOffline, } d, _, err := resolver.ResolveExistingObject(ctx, schemaResolver, uon, lookupFlags) if err != nil { From 6ab314ca86ad8c5c3f89f426dec5ec451bfe5a93 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Wed, 8 Jan 2025 17:57:51 -0500 Subject: [PATCH 036/126] crosscluster/producer: allow LDR creation from offline tables This patch allows a user to create an LDR stream that replicates from an offline table. Epic: none Release note: none --- pkg/crosscluster/producer/replication_manager.go | 2 +- pkg/repstream/streampb/stream.proto | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/crosscluster/producer/replication_manager.go b/pkg/crosscluster/producer/replication_manager.go index f92f82019ceb..781e5e0d8789 100644 --- a/pkg/crosscluster/producer/replication_manager.go +++ b/pkg/crosscluster/producer/replication_manager.go @@ -83,7 +83,7 @@ func (r *replicationStreamManagerImpl) StartReplicationStreamForTables( mutableTableDescs := make([]*tabledesc.Mutable, 0, len(req.TableNames)) tableIDs := make([]uint32, 0, len(req.TableNames)) - externalCatalog, err := externalcatalog.ExtractExternalCatalog(ctx, r.resolver, r.txn, r.txn.Descriptors(), false /* includeOffline */, req.TableNames...) + externalCatalog, err := externalcatalog.ExtractExternalCatalog(ctx, r.resolver, r.txn, r.txn.Descriptors(), req.AllowOffline, req.TableNames...) if err != nil { return streampb.ReplicationProducerSpec{}, err } diff --git a/pkg/repstream/streampb/stream.proto b/pkg/repstream/streampb/stream.proto index de26310a5da1..44fa4eafd009 100644 --- a/pkg/repstream/streampb/stream.proto +++ b/pkg/repstream/streampb/stream.proto @@ -83,6 +83,8 @@ message ReplicationProducerRequest { // TableNames, if set, are the names of the individual tables that a // logical replication ingestion processor are interested in. repeated string table_names = 4; + + bool allow_offline = 5; } enum ReplicationType { From 15876c2260683b049249052148e35bc2f4823722 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Wed, 8 Jan 2025 17:59:08 -0500 Subject: [PATCH 037/126] crosscluster/logical: enable automatic bidi replication This patch enables a user to create a bidirectional replication stream with one command. When the user runs `CREATE LOGICALLY REPLICATED TABLE tabB FROM tabA ON uriA WITH BIDIRECTIONAL ON uriB`, the following will happen: - on cluster B, during job planning, construct the reverse stream cmd, while we still have access to original LDR cmd. This command will have the following form: `CREATE LOGICAL REPLICATION STREAM FROM TABLE tabB ON 'uriB' INTO TABLE tabA WITH CURSOR = $1, PARENT = '{og job id}';` - on cluster B, create table tabB and begin an offline scan, as normal - on cluster B, once the offline scan completes, but before tabB is published, set up the reverse stream from B to A at a cursor time after the initial scan completes but before the tabB has come online. This cursor time prevents data looping and ensures the replication of all future foreground writes. - on cluster B, steady state replication begins. Epic: none Release note (sql change): when a user runs CREATE LOGICALLY REPLICATED TABLE, they must specify one of the following options: - UNIDIRECTIONAL: setups a unidirectional stream with fast initial scan - BIDIRECTIONAL ON {dest uri}: sets up a bidirectional stream from the original dest to the original source. --- pkg/crosscluster/logical/BUILD.bazel | 1 + .../create_logical_replication_stmt.go | 70 +++++++++++++++++-- .../logical/logical_replication_job.go | 62 +++++++++++++--- .../logical/logical_replication_job_test.go | 43 +++++++++++- pkg/jobs/jobspb/jobs.proto | 21 +++++- 5 files changed, 176 insertions(+), 21 deletions(-) diff --git a/pkg/crosscluster/logical/BUILD.bazel b/pkg/crosscluster/logical/BUILD.bazel index 8a7c102dc703..e84738aa1213 100644 --- a/pkg/crosscluster/logical/BUILD.bazel +++ b/pkg/crosscluster/logical/BUILD.bazel @@ -44,6 +44,7 @@ go_library( "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog", + "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", diff --git a/pkg/crosscluster/logical/create_logical_replication_stmt.go b/pkg/crosscluster/logical/create_logical_replication_stmt.go index 29e5eaa7da57..43c8ded1542a 100644 --- a/pkg/crosscluster/logical/create_logical_replication_stmt.go +++ b/pkg/crosscluster/logical/create_logical_replication_stmt.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" @@ -100,7 +101,7 @@ func createLogicalReplicationStreamPlanHook( return pgerror.New(pgcode.InvalidParameterValue, "the same number of source and destination tables must be specified") } - options, err := evalLogicalReplicationOptions(ctx, stmt.Options, exprEval, p) + options, err := evalLogicalReplicationOptions(ctx, stmt.Options, exprEval, p, stmt.CreateTable) if err != nil { return err } @@ -135,6 +136,7 @@ func createLogicalReplicationStreamPlanHook( return pgerror.Newf(pgcode.InvalidParameterValue, "unknown discard option %q", m) } } + resolvedDestObjects, err := resolveDestinationObjects(ctx, p, p.SessionData(), stmt.Into, stmt.CreateTable) if err != nil { return err @@ -188,7 +190,8 @@ func createLogicalReplicationStreamPlanHook( srcTableNames[i] = tb.String() } spec, err := client.CreateForTables(ctx, &streampb.ReplicationProducerRequest{ - TableNames: srcTableNames, + TableNames: srcTableNames, + AllowOffline: options.ParentID != 0, }) if err != nil { return err @@ -253,8 +256,23 @@ func createLogicalReplicationStreamPlanHook( defaultConflictResolution = *cr } + jobID := p.ExecCfg().JobRegistry.MakeJobID() + var reverseStreamCmd string + if stmt.CreateTable && options.BidirectionalURI() != "" { + // TODO: validate URI. + + reverseStmt := *stmt + reverseStmt.From, reverseStmt.Into = reverseStmt.Into, reverseStmt.From + reverseStmt.CreateTable = false + reverseStmt.Options.BidirectionalURI = nil + reverseStmt.Options.ParentID = tree.NewStrVal(jobID.String()) + reverseStmt.PGURL = tree.NewStrVal(options.BidirectionalURI()) + reverseStmt.Options.Cursor = &tree.Placeholder{Idx: 0} + reverseStreamCmd = reverseStmt.String() + } + jr := jobs.Record{ - JobID: p.ExecCfg().JobRegistry.MakeJobID(), + JobID: jobID, Description: fmt.Sprintf("LOGICAL REPLICATION STREAM into %s from %s", resolvedDestObjects.TargetDescription(), cleanedURI), Username: p.User(), Details: jobspb.LogicalReplicationDetails{ @@ -269,6 +287,8 @@ func createLogicalReplicationStreamPlanHook( Mode: mode, MetricsLabel: options.metricsLabel, CreateTable: stmt.CreateTable, + ReverseStreamCommand: reverseStreamCmd, + ParentID: int64(options.ParentID), }, Progress: progress, } @@ -448,9 +468,13 @@ func createLogicalReplicationStreamTypeCheck( stmt.Options.Mode, stmt.Options.MetricsLabel, stmt.Options.Discard, + stmt.Options.BidirectionalURI, + stmt.Options.ParentID, }, + exprutil.Ints{stmt.Options.ParentID}, exprutil.Bools{ stmt.Options.SkipSchemaCheck, + stmt.Options.Unidirectional, }, } if err := exprutil.TypeCheck(ctx, "LOGICAL REPLICATION STREAM", p.SemaCtx(), @@ -467,10 +491,12 @@ type resolvedLogicalReplicationOptions struct { mode string defaultFunction *jobspb.LogicalReplicationDetails_DefaultConflictResolution // Mapping of table name to function descriptor - userFunctions map[string]int32 - discard string - skipSchemaCheck bool - metricsLabel string + userFunctions map[string]int32 + discard string + skipSchemaCheck bool + metricsLabel string + bidirectionalURI string + ParentID catpb.JobID } func evalLogicalReplicationOptions( @@ -478,6 +504,7 @@ func evalLogicalReplicationOptions( options tree.LogicalReplicationOptions, eval exprutil.Evaluator, p sql.PlanHookState, + createTable bool, ) (*resolvedLogicalReplicationOptions, error) { r := &resolvedLogicalReplicationOptions{} if options.Mode != nil { @@ -562,6 +589,28 @@ func evalLogicalReplicationOptions( if options.SkipSchemaCheck == tree.DBoolTrue { r.skipSchemaCheck = true } + if options.ParentID != nil { + parentID, err := eval.Int(ctx, options.ParentID) + if err != nil { + return nil, err + } + r.ParentID = catpb.JobID(parentID) + } + unidirectional := options.Unidirectional == tree.DBoolTrue + + if options.BidirectionalURI != nil { + uri, err := eval.String(ctx, options.BidirectionalURI) + if err != nil { + return nil, err + } + r.bidirectionalURI = uri + } + if createTable && unidirectional && r.bidirectionalURI != "" { + return nil, errors.New("UNIDIRECTIONAL and BIDIRECTIONAL cannot be specified together") + } + if createTable && !unidirectional && r.bidirectionalURI == "" { + return nil, errors.New("either BIDIRECTIONAL or UNIDRECTIONAL must be specified") + } return r, nil } @@ -627,3 +676,10 @@ func (r *resolvedLogicalReplicationOptions) SkipSchemaCheck() bool { } return r.skipSchemaCheck } + +func (r *resolvedLogicalReplicationOptions) BidirectionalURI() string { + if r == nil || r.bidirectionalURI == "" { + return "" + } + return r.bidirectionalURI +} diff --git a/pkg/crosscluster/logical/logical_replication_job.go b/pkg/crosscluster/logical/logical_replication_job.go index 8c02f4dbf682..467c8d4b985d 100644 --- a/pkg/crosscluster/logical/logical_replication_job.go +++ b/pkg/crosscluster/logical/logical_replication_job.go @@ -150,10 +150,6 @@ func (r *logicalReplicationResumer) ingest( replicatedTimeAtStart = progress.ReplicatedTime ) - if err := r.maybePublishCreatedTables(ctx, jobExecCtx, progress, payload); err != nil { - return err - } - uris, err := r.getClusterUris(ctx, r.job, execCfg.InternalDB) if err != nil { return err @@ -170,6 +166,14 @@ func (r *logicalReplicationResumer) ingest( } defer func() { _ = client.Close(ctx) }() + if err := r.maybeStartReverseStream(ctx, jobExecCtx, client); err != nil { + return err + } + + if err := r.maybePublishCreatedTables(ctx, jobExecCtx); err != nil { + return err + } + asOf := replicatedTimeAtStart if asOf.IsEmpty() { asOf = payload.ReplicationStartTime @@ -194,6 +198,8 @@ func (r *logicalReplicationResumer) ingest( }); err != nil { return err } + // Update the local progress copy as it was just updated. + progress = r.job.Progress().Details.(*jobspb.Progress_LogicalReplication).LogicalReplication // TODO(azhu): add a flag to avoid recreating dlq tables during replanning if !(payload.CreateTable && progress.ReplicatedTime.IsEmpty()) { @@ -295,15 +301,55 @@ func (r *logicalReplicationResumer) ingest( return err } +func (r *logicalReplicationResumer) maybeStartReverseStream( + ctx context.Context, jobExecCtx sql.JobExecContext, client streamclient.Client, +) error { + + // Instantiate a local copy of progress and details as they are gated behind a mutex. + progress := r.job.Progress().Details.(*jobspb.Progress_LogicalReplication).LogicalReplication + details := r.job.Details().(jobspb.LogicalReplicationDetails) + + if !(details.ReverseStreamCommand != "" && progress.ReplicatedTime.IsSet() && !progress.StartedReverseStream) { + return nil + } + + // Begin the reverse stream at a system time before source tables have been + // published but after they have been created during this job's planning. + now := jobExecCtx.ExecCfg().Clock.Now() + if err := client.ExecStatement(ctx, details.ReverseStreamCommand, "start-reverse-stream", now.AsOfSystemTime()); err != nil { + return errors.Wrapf(err, "failed to start reverse stream") + } + + // TODO(msbutler): if the job exits before we write here but after setting up + // the reverse stream, we will accidentally create a second reverse stream. To + // prevent this, the PARENT option will passed to the reverse stream, then + // during ldr stream creation, the planhook checks if any job is already + // running with this parent job id. + if err := r.job.NoTxn().Update(ctx, func(txn isql.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + md.Progress.Details.(*jobspb.Progress_LogicalReplication).LogicalReplication.StartedReverseStream = true + ju.UpdateProgress(md.Progress) + return nil + }); err != nil { + return err + } + log.Infof(ctx, "started reverse stream") + return nil +} + func (r *logicalReplicationResumer) maybePublishCreatedTables( - ctx context.Context, - jobExecCtx sql.JobExecContext, - progress *jobspb.LogicalReplicationProgress, - details jobspb.LogicalReplicationDetails, + ctx context.Context, jobExecCtx sql.JobExecContext, ) error { + + // Instantiate a local copy of progress and details as they are gated behind a mutex. + progress := r.job.Progress().Details.(*jobspb.Progress_LogicalReplication).LogicalReplication + details := r.job.Details().(jobspb.LogicalReplicationDetails) + if !(details.CreateTable && progress.ReplicatedTime.IsSet() && !progress.PublishedNewTables) { return nil } + if details.ReverseStreamCommand != "" && !progress.StartedReverseStream { + return errors.AssertionFailedf("attempting to publish descriptors before starting reverse stream") + } return sql.DescsTxn(ctx, jobExecCtx.ExecCfg(), func(ctx context.Context, txn isql.Txn, descCol *descs.Collection) error { b := txn.KV().NewBatch() for i := range details.IngestedExternalCatalog.Tables { diff --git a/pkg/crosscluster/logical/logical_replication_job_test.go b/pkg/crosscluster/logical/logical_replication_job_test.go index 57e35f9d9efe..6de3ed461f2b 100644 --- a/pkg/crosscluster/logical/logical_replication_job_test.go +++ b/pkg/crosscluster/logical/logical_replication_job_test.go @@ -381,6 +381,7 @@ func TestLogicalStreamIngestionJobWithCursor(t *testing.T) { func TestCreateTables(t *testing.T) { defer leaktest.AfterTest(t)() skip.UnderDeadlock(t) + skip.UnderRace(t) defer log.Scope(t).Close(t) ctx := context.Background() @@ -403,7 +404,7 @@ func TestCreateTables(t *testing.T) { sqlB := sqlutils.MakeSQLRunner(srv.SQLConn(t, serverutils.DBName("b"))) var jobID jobspb.JobID - sqlB.QueryRow(t, "CREATE LOGICALLY REPLICATED TABLE b.tab FROM TABLE tab ON $1", aURL.String()).Scan(&jobID) + sqlB.QueryRow(t, "CREATE LOGICALLY REPLICATED TABLE b.tab FROM TABLE tab ON $1 WITH UNIDIRECTIONAL", aURL.String()).Scan(&jobID) // Check LWW on initial scan data. sqlA.Exec(t, "UPSERT INTO tab VALUES (1, 'howdy')") @@ -416,14 +417,18 @@ func TestCreateTables(t *testing.T) { // Ensure secondary index was replicated as well. compareReplicatedTables(t, srv, "a", "b", "tab", sqlA, sqlB) }) + t.Run("pause initial scan", func(t *testing.T) { sqlA.Exec(t, "CREATE DATABASE c") sqlA.Exec(t, "CREATE TABLE tab2 (pk int primary key, payload string)") sqlc := sqlutils.MakeSQLRunner(srv.SQLConn(t, serverutils.DBName("c"))) sqlc.Exec(t, "SET CLUSTER SETTING jobs.debug.pausepoints = 'logical_replication.after.retryable_error'") + defer func() { + sqlc.Exec(t, "RESET CLUSTER SETTING jobs.debug.pausepoints") + }() var jobID jobspb.JobID - sqlc.QueryRow(t, "CREATE LOGICALLY REPLICATED TABLE tab2 FROM TABLE tab2 ON $1", aURL.String()).Scan(&jobID) + sqlc.QueryRow(t, "CREATE LOGICALLY REPLICATED TABLE tab2 FROM TABLE tab2 ON $1 WITH UNIDIRECTIONAL", aURL.String()).Scan(&jobID) jobutils.WaitForJobToPause(t, sqlc, jobID) // Verify created tables are not visible as we paused before publishing @@ -450,13 +455,45 @@ func TestCreateTables(t *testing.T) { // rerun LDR again. As you can see in the // restore-on-fail-or-cancel-fast-drop test, setting this up is a pain, so I // will address this in an upcoming pr. - sqlc.QueryRow(t, "CREATE LOGICALLY REPLICATED TABLE tab FROM TABLE tab ON $1", aURL.String()).Scan(&jobID) + sqlc.QueryRow(t, "CREATE LOGICALLY REPLICATED TABLE tab FROM TABLE tab ON $1 WITH UNIDIRECTIONAL", aURL.String()).Scan(&jobID) jobutils.WaitForJobToPause(t, sqlc, jobID) // Next, resume it and wait for the table and its dlq table to come online. sqlc.Exec(t, "RESUME JOB $1", jobID) sqlc.CheckQueryResultsRetry(t, "SELECT count(*) FROM [SHOW TABLES]", [][]string{{"2"}}) }) + t.Run("bidi", func(t *testing.T) { + sqlA.Exec(t, "CREATE TABLE tab3 (pk int primary key, payload string)") + sqlA.Exec(t, "INSERT INTO tab3 VALUES (1, 'hello')") + + sqlA.Exec(t, "CREATE DATABASE d") + sqlD := sqlutils.MakeSQLRunner(srv.SQLConn(t, serverutils.DBName("d"))) + dURL, cleanup := srv.PGUrl(t, serverutils.DBName("d")) + defer cleanup() + + var jobID jobspb.JobID + sqlD.QueryRow(t, "CREATE LOGICALLY REPLICATED TABLE tab3 FROM TABLE tab3 ON $1 WITH BIDIRECTIONAL ON $2", aURL.String(), dURL.String()).Scan(&jobID) + WaitUntilReplicatedTime(t, srv.Clock().Now(), sqlD, jobID) + sqlD.CheckQueryResultsRetry(t, `SELECT count(*) FROM [SHOW LOGICAL REPLICATION JOBS] WHERE tables = '{a.public.tab3}'`, [][]string{{"1"}}) + var reverseJobID jobspb.JobID + sqlD.QueryRow(t, "SELECT job_id FROM [SHOW LOGICAL REPLICATION JOBS] WHERE tables = '{a.public.tab3}'").Scan(&reverseJobID) + sqlD.Exec(t, "INSERT INTO tab3 VALUES (2, 'goodbye')") + sqlA.Exec(t, "INSERT INTO tab3 VALUES (3, 'brb')") + WaitUntilReplicatedTime(t, srv.Clock().Now(), sqlD, jobID) + WaitUntilReplicatedTime(t, srv.Clock().Now(), sqlD, reverseJobID) + compareReplicatedTables(t, srv, "a", "d", "tab3", sqlA, sqlD) + }) + t.Run("create command errors", func(t *testing.T) { + sqlA.Exec(t, "CREATE TABLE tab4 (pk int primary key, payload string)") + sqlA.Exec(t, "INSERT INTO tab4 VALUES (1, 'hello')") + + sqlA.Exec(t, "CREATE DATABASE e") + sqlE := sqlutils.MakeSQLRunner(srv.SQLConn(t, serverutils.DBName("e"))) + eURL, cleanup := srv.PGUrl(t, serverutils.DBName("e")) + defer cleanup() + sqlE.ExpectErr(t, "either BIDIRECTIONAL or UNIDRECTIONAL must be specified", "CREATE LOGICALLY REPLICATED TABLE b.tab4 FROM TABLE tab4 ON $1", eURL.String()) + sqlE.ExpectErr(t, "UNIDIRECTIONAL and BIDIRECTIONAL cannot be specified together", "CREATE LOGICALLY REPLICATED TABLE tab4 FROM TABLE tab4 ON $1 WITH BIDIRECTIONAL ON $2, UNIDIRECTIONAL", aURL.String(), eURL.String()) + }) } // TestLogicalStreamIngestionAdvancePTS tests that the producer side pts advances diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index cacae1048d15..da90ffb04b56 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -272,12 +272,25 @@ message LogicalReplicationDetails { } Discard discard = 11; - + + // CreateTable is true if the job should create the table(s) in the + // destination. bool create_table = 12; - + + // IngestedExternalCatalog is the catalog written to the destination cluster + // when CreateTable is true. sql.catalog.externalcatalog.externalpb.ExternalCatalog ingested_external_catalog = 13 [(gogoproto.nullable) = false]; + + // ReverseStreamCommand is CREATE LDR command the coordinator will issue once + // the offline initial scan completes, but before the tables are made public. + string reverse_stream_command = 14; + + // ParentID is set on the reverse stream job in automatic bidirectional + // replication, and is equal to the job ID that issued the reverse stream + // command. + int64 parent_id = 15 [(gogoproto.customname) = "ParentID"]; - // Next ID 14; + // Next ID: 16. } message LogicalReplicationProgress { @@ -296,6 +309,8 @@ message LogicalReplicationProgress { repeated string partition_conn_uris = 8; bool published_new_tables = 9; + + bool started_reverse_stream = 10; } message StreamReplicationDetails { From b1f9f7bc096d36508a91fe9011a2b147a864f11b Mon Sep 17 00:00:00 2001 From: rharding6373 Date: Fri, 10 Jan 2025 15:26:07 -0800 Subject: [PATCH 038/126] tests: get cdc_bench cursor at least 1s after table creation Previously, we would get changefeed cursors from the current time after creating a table and before populating it. Due to the time being at a second granularity, it was possible that the cursor could be the same second that the table was created, which would lead to the table not appearing to be created yet. This change adds a 1s sleep before getting the cursor timestamp, to ensure that we don't encounter this scenario. Epic: None Fixes: #137758 Fixes: #135795 Release note: None --- pkg/cmd/roachtest/tests/cdc_bench.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/cmd/roachtest/tests/cdc_bench.go b/pkg/cmd/roachtest/tests/cdc_bench.go index fd17c9ea067f..b33c7af80642 100644 --- a/pkg/cmd/roachtest/tests/cdc_bench.go +++ b/pkg/cmd/roachtest/tests/cdc_bench.go @@ -274,6 +274,7 @@ func runCDCBenchScan( `./cockroach workload init kv --splits %d {pgurl:%d}`, numRanges, nData[0])) require.NoError(t, roachtestutil.WaitFor3XReplication(ctx, t.L(), conn)) + time.Sleep(1 * time.Second) cursor := timeutil.Now() // before data is ingested // Ingest data. init allows us to import into the existing table. However, From 998ba728373a8525d29304b15def8762b801b9fb Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Fri, 10 Jan 2025 21:35:37 -0800 Subject: [PATCH 039/126] cloudtestutils: refactor CheckExportStore This change adds a `cloudtestutils.StoreInfo` structure and refactors `CheckExportStore` to use it, making the call sites simpler and more readable. We now pass the external IO dir explicitly through `StoreInfo` instead of getting it from the settings. Epic: none Release note: None --- pkg/cloud/amazon/s3_storage_test.go | 147 +++++++++--------- pkg/cloud/azure/azure_storage_test.go | 35 ++--- .../cloudtestutils/cloud_test_helpers.go | 54 ++++--- pkg/cloud/gcp/gcs_storage_test.go | 88 +++++------ pkg/cloud/httpsink/http_storage_test.go | 20 ++- pkg/cloud/nodelocal/nodelocal_storage_test.go | 8 +- pkg/cloud/userfile/file_table_storage_test.go | 12 +- 7 files changed, 190 insertions(+), 174 deletions(-) diff --git a/pkg/cloud/amazon/s3_storage_test.go b/pkg/cloud/amazon/s3_storage_test.go index e3c12b6f426e..4e6de9191ee9 100644 --- a/pkg/cloud/amazon/s3_storage_test.go +++ b/pkg/cloud/amazon/s3_storage_test.go @@ -106,14 +106,10 @@ func TestPutS3(t *testing.T) { } t.Run(testName, func(t *testing.T) { t.Run("auth-empty-no-cred", func(t *testing.T) { - _, err := cloud.ExternalStorageFromURI(ctx, fmt.Sprintf("s3://%s/%s-%d", bucket, - "backup-test-default", testID), base.ExternalIODirConfig{}, testSettings, - blobs.TestEmptyBlobClientFactory, user, - nil, /* ie */ - nil, /* ief */ - nil, /* kvDB */ - nil, /* limiters */ - nil, /* metrics */ + uri := fmt.Sprintf("s3://%s/%s-%d", bucket, "backup-test-default", testID) + _, err := cloud.ExternalStorageFromURI( + ctx, uri, base.ExternalIODirConfig{}, testSettings, blobs.TestEmptyBlobClientFactory, user, + nil /* ie */, nil /* ief */, nil /* kvDB */, nil /* limiters */, nil, /* metrics */ ) require.EqualError(t, err, fmt.Sprintf( `%s is set to '%s', but %s is not set`, @@ -124,55 +120,54 @@ func TestPutS3(t *testing.T) { }) t.Run("auth-implicit", func(t *testing.T) { skipIfNoDefaultConfig(t, ctx) - cloudtestutils.CheckExportStore(t, fmt.Sprintf( - "s3://%s/%s-%d?%s=%s", - bucket, "backup-test-default", testID, - cloud.AuthParam, cloud.AuthParamImplicit, - ), false, user, - nil, /* db */ - testSettings) + info := cloudtestutils.StoreInfo{ + URI: fmt.Sprintf( + "s3://%s/%s-%d?%s=%s", + bucket, "backup-test-default", testID, + cloud.AuthParam, cloud.AuthParamImplicit, + ), + User: user, + } + cloudtestutils.CheckExportStore(t, info) }) t.Run("auth-specified", func(t *testing.T) { - uri := S3URI(bucket, fmt.Sprintf("backup-test-%d", testID), - &cloudpb.ExternalStorage_S3{AccessKey: envCreds.AccessKeyID, Secret: envCreds.SecretAccessKey, Region: "us-east-1"}, - ) - cloudtestutils.CheckExportStore( - t, uri, false, user, nil /* db */, testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: S3URI( + bucket, fmt.Sprintf("backup-test-%d", testID), + &cloudpb.ExternalStorage_S3{AccessKey: envCreds.AccessKeyID, Secret: envCreds.SecretAccessKey, Region: "us-east-1"}, + ), + User: user, + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles( - t, uri, user, nil /* db */, testSettings, + t, info.URI, info.User, nil /* db */, testSettings, ) }) // Tests that we can put an object with server side encryption specified. t.Run("server-side-encryption", func(t *testing.T) { skipIfNoDefaultConfig(t, ctx) - cloudtestutils.CheckExportStore(t, fmt.Sprintf( - "s3://%s/%s-%d?%s=%s&%s=%s", - bucket, "backup-test-sse-256", testID, - cloud.AuthParam, cloud.AuthParamImplicit, AWSServerSideEncryptionMode, - "AES256", - ), - false, - user, - nil, /* db */ - testSettings, - ) - + info := cloudtestutils.StoreInfo{ + URI: fmt.Sprintf( + "s3://%s/%s-%d?%s=%s&%s=%s", + bucket, "backup-test-sse-256", testID, + cloud.AuthParam, cloud.AuthParamImplicit, AWSServerSideEncryptionMode, + "AES256", + ), + User: user, + } + cloudtestutils.CheckExportStore(t, info) v := os.Getenv("AWS_KMS_KEY_ARN") if v == "" { skip.IgnoreLint(t, "AWS_KMS_KEY_ARN env var must be set") } - cloudtestutils.CheckExportStore(t, fmt.Sprintf( + info.URI = fmt.Sprintf( "s3://%s/%s-%d?%s=%s&%s=%s&%s=%s", bucket, "backup-test-sse-kms", testID, cloud.AuthParam, cloud.AuthParamImplicit, AWSServerSideEncryptionMode, "aws:kms", AWSServerSideEncryptionKMSID, v, - ), - false, - user, - nil, /* db */ - testSettings) + ) + cloudtestutils.CheckExportStore(t, info) }) t.Run("server-side-encryption-invalid-params", func(t *testing.T) { @@ -197,7 +192,6 @@ func TestPutS3(t *testing.T) { require.True(t, testutils.IsError(err, "AWS_SERVER_KMS_ID param must be set when using aws:kms server side encryption mode.")) }) }) - } } @@ -231,26 +225,28 @@ func TestPutS3AssumeRole(t *testing.T) { ctx := context.Background() t.Run("auth-implicit", func(t *testing.T) { skipIfNoDefaultConfig(t, ctx) - uri := S3URI(bucket, testPath, - &cloudpb.ExternalStorage_S3{Auth: cloud.AuthParamImplicit, RoleARN: roleArn, Region: "us-east-1"}, - ) - cloudtestutils.CheckExportStore( - t, uri, false, user, nil /* db */, testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: S3URI(bucket, testPath, + &cloudpb.ExternalStorage_S3{Auth: cloud.AuthParamImplicit, RoleARN: roleArn, Region: "us-east-1"}, + ), + User: user, + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles( - t, uri, user, nil /* db */, testSettings, + t, info.URI, user, nil /* db */, testSettings, ) }) t.Run("auth-specified", func(t *testing.T) { - uri := S3URI(bucket, testPath, - &cloudpb.ExternalStorage_S3{Auth: cloud.AuthParamSpecified, RoleARN: roleArn, AccessKey: creds.AccessKeyID, Secret: creds.SecretAccessKey, Region: "us-east-1"}, - ) - cloudtestutils.CheckExportStore( - t, uri, false, user, nil /* db */, testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: S3URI(bucket, testPath, + &cloudpb.ExternalStorage_S3{Auth: cloud.AuthParamSpecified, RoleARN: roleArn, AccessKey: creds.AccessKeyID, Secret: creds.SecretAccessKey, Region: "us-east-1"}, + ), + User: user, + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles( - t, uri, user, nil /* db */, testSettings, + t, info.URI, user, nil /* db */, testSettings, ) }) @@ -295,22 +291,25 @@ func TestPutS3AssumeRole(t *testing.T) { delegatesWithoutID = append(delegatesWithoutID, cloudpb.ExternalStorage_AssumeRoleProvider{Role: p.Role}) } - uri := S3URI(bucket, testPath, - &cloudpb.ExternalStorage_S3{ - Auth: tc.auth, - AssumeRoleProvider: roleWithoutID, - DelegateRoleProviders: delegatesWithoutID, - AccessKey: tc.accessKey, - Secret: tc.secretKey, - Region: "us-east-1", - }, - ) + info := cloudtestutils.StoreInfo{ + URI: S3URI(bucket, testPath, + &cloudpb.ExternalStorage_S3{ + Auth: tc.auth, + AssumeRoleProvider: roleWithoutID, + DelegateRoleProviders: delegatesWithoutID, + AccessKey: tc.accessKey, + Secret: tc.secretKey, + Region: "us-east-1", + }, + ), + User: user, + } cloudtestutils.CheckNoPermission( - t, uri, user, nil /* db */, testSettings, + t, info.URI, user, nil /* db */, testSettings, ) // Finally, check that the chain of roles can be used to access the storage. - uri = S3URI(bucket, testPath, + info.URI = S3URI(bucket, testPath, &cloudpb.ExternalStorage_S3{ Auth: tc.auth, AssumeRoleProvider: providerChain[len(providerChain)-1], @@ -321,9 +320,7 @@ func TestPutS3AssumeRole(t *testing.T) { }, ) - cloudtestutils.CheckExportStore( - t, uri, false, user, nil /* db */, testSettings, - ) + cloudtestutils.CheckExportStore(t, info) }) } }) @@ -363,11 +360,11 @@ func TestPutS3Endpoint(t *testing.T) { RawQuery: q.Encode(), } - testSettings := cluster.MakeTestingClusterSettings() - - cloudtestutils.CheckExportStore( - t, u.String(), false, user, nil /* db */, testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: u.String(), + User: user, + } + cloudtestutils.CheckExportStore(t, info) }) t.Run("use-path-style", func(t *testing.T) { // EngFlow machines have no internet access, and queries even to localhost will time out. diff --git a/pkg/cloud/azure/azure_storage_test.go b/pkg/cloud/azure/azure_storage_test.go index b2903df804ac..465815fe1431 100644 --- a/pkg/cloud/azure/azure_storage_test.go +++ b/pkg/cloud/azure/azure_storage_test.go @@ -109,11 +109,11 @@ func TestAzure(t *testing.T) { testPath := fmt.Sprintf("backup-test-%d", testID) testListPath := fmt.Sprintf("listing-test-%d", testID) - cloudtestutils.CheckExportStore(t, cfg.filePath(testPath), - false, username.RootUserName(), - nil, /* db */ - testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: cfg.filePath(testPath), + User: username.RootUserName(), + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, cfg.filePath(testListPath), username.RootUserName(), nil, /* db */ @@ -121,11 +121,8 @@ func TestAzure(t *testing.T) { ) // Client Secret auth - cloudtestutils.CheckExportStore(t, cfg.filePathClientAuth(testPath), - false, username.RootUserName(), - nil, /* db */ - testSettings, - ) + info.URI = cfg.filePathClientAuth(testPath) + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, cfg.filePathClientAuth(testListPath), username.RootUserName(), nil, /* db */ @@ -133,11 +130,8 @@ func TestAzure(t *testing.T) { ) // Implicit auth - cloudtestutils.CheckExportStore(t, cfg.filePathImplicitAuth(testPath), - false, username.RootUserName(), - nil, /* db */ - testSettings, - ) + info.URI = cfg.filePathImplicitAuth(testPath) + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, cfg.filePathImplicitAuth(testListPath), username.RootUserName(), nil, /* db */ @@ -300,11 +294,12 @@ func TestAzureStorageFileImplicitAuth(t *testing.T) { cleanup3 := envutil.TestSetEnv(t, "COCKROACH_AZURE_APPLICATION_CREDENTIALS_FILE", credFile) defer cleanup3() - cloudtestutils.CheckExportStore(t, cfg.filePathImplicitAuth(testPath), - false, username.RootUserName(), - nil, /* db */ - testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: cfg.filePathImplicitAuth(testPath), + User: username.RootUserName(), + } + + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, cfg.filePathImplicitAuth(testListPath), username.RootUserName(), nil, /* db */ diff --git a/pkg/cloud/cloudtestutils/cloud_test_helpers.go b/pkg/cloud/cloudtestutils/cloud_test_helpers.go index 5df0c001dfd3..b8bf85b64dae 100644 --- a/pkg/cloud/cloudtestutils/cloud_test_helpers.go +++ b/pkg/cloud/cloudtestutils/cloud_test_helpers.go @@ -118,27 +118,45 @@ func storeFromURI( return s } -// CheckExportStore runs an array of tests against a storeURI. -func CheckExportStore( - t *testing.T, - storeURI string, - skipSingleFile bool, - user username.SQLUsername, - db isql.DB, - testSettings *cluster.Settings, -) { +type StoreInfo struct { + URI string + User username.SQLUsername + + // Fields below are optional. + + TestSettings *cluster.Settings + DB isql.DB + ExternalIODir string +} + +// CheckExportStore runs an array of tests against a store. +func CheckExportStore(t *testing.T, info StoreInfo) { + checkExportStore(t, info, false /* skipSingleFile */) +} + +// CheckExportStoreSkipSingleFile runs an array of tests against a store, +// skipping single file tests. +func CheckExportStoreSkipSingleFile(t *testing.T, info StoreInfo) { + checkExportStore(t, info, true /* skipSingleFile */) +} + +// CheckExportStore runs an array of tests against a store. +func checkExportStore(t *testing.T, info StoreInfo, skipSingleFile bool) { ioConf := base.ExternalIODirConfig{} ctx := context.Background() - conf, err := cloud.ExternalStorageConfFromURI(storeURI, user) + if info.TestSettings == nil { + info.TestSettings = cluster.MakeTestingClusterSettings() + } + + conf, err := cloud.ExternalStorageConfFromURI(info.URI, info.User) if err != nil { t.Fatal(err) } // Setup a sink for the given args. - clientFactory := blobs.TestBlobServiceClient(testSettings.ExternalIODir) - s, err := cloud.MakeExternalStorage(ctx, conf, ioConf, testSettings, clientFactory, - db, nil, cloud.NilMetrics) + clientFactory := blobs.TestBlobServiceClient(info.ExternalIODir) + s, err := cloud.MakeExternalStorage(ctx, conf, ioConf, info.TestSettings, clientFactory, info.DB, nil, cloud.NilMetrics) if err != nil { t.Fatal(err) } @@ -249,8 +267,8 @@ func CheckExportStore( if err := cloud.WriteFile(ctx, s, testingFilename, bytes.NewReader([]byte("aaa"))); err != nil { t.Fatal(err) } - singleFile := storeFromURI(ctx, t, appendPath(t, storeURI, testingFilename), clientFactory, - user, db, testSettings) + singleFile := storeFromURI(ctx, t, appendPath(t, info.URI, testingFilename), clientFactory, + info.User, info.DB, info.TestSettings) defer singleFile.Close() res, _, err := singleFile.ReadFile(ctx, "", cloud.ReadOptions{NoFileSize: true}) @@ -270,8 +288,8 @@ func CheckExportStore( }) t.Run("write-single-file-by-uri", func(t *testing.T) { const testingFilename = "B" - singleFile := storeFromURI(ctx, t, appendPath(t, storeURI, testingFilename), clientFactory, - user, db, testSettings) + singleFile := storeFromURI(ctx, t, appendPath(t, info.URI, testingFilename), clientFactory, + info.User, info.DB, info.TestSettings) defer singleFile.Close() if err := cloud.WriteFile(ctx, singleFile, "", bytes.NewReader([]byte("bbb"))); err != nil { @@ -302,7 +320,7 @@ func CheckExportStore( if err := cloud.WriteFile(ctx, s, testingFilename, bytes.NewReader([]byte("aaa"))); err != nil { t.Fatal(err) } - singleFile := storeFromURI(ctx, t, storeURI, clientFactory, user, db, testSettings) + singleFile := storeFromURI(ctx, t, info.URI, clientFactory, info.User, info.DB, info.TestSettings) defer singleFile.Close() // Read a valid file. diff --git a/pkg/cloud/gcp/gcs_storage_test.go b/pkg/cloud/gcp/gcs_storage_test.go index ddc7350d7ed3..aba83e360a55 100644 --- a/pkg/cloud/gcp/gcs_storage_test.go +++ b/pkg/cloud/gcp/gcs_storage_test.go @@ -58,14 +58,11 @@ func TestPutGoogleCloud(t *testing.T) { if specified { uri += fmt.Sprintf("&%s=%s", cloud.AuthParam, cloud.AuthParamSpecified) } - cloudtestutils.CheckExportStore( - t, - uri, - false, - user, - nil, /* db */ - testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: uri, + User: user, + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", bucket, "backup-test-specified", @@ -84,15 +81,12 @@ func TestPutGoogleCloud(t *testing.T) { if !cloudtestutils.IsImplicitAuthConfigured() { skip.IgnoreLint(t, "implicit auth is not configured") } - - cloudtestutils.CheckExportStore( - t, - fmt.Sprintf("gs://%s/%s-%d?%s=%s", bucket, "backup-test-implicit", testID, + info := cloudtestutils.StoreInfo{ + URI: fmt.Sprintf("gs://%s/%s-%d?%s=%s", bucket, "backup-test-implicit", testID, cloud.AuthParam, cloud.AuthParamImplicit), - false, - user, - nil, /* db */ - testSettings) + User: user, + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s", bucket, "backup-test-implicit", @@ -128,13 +122,11 @@ func TestPutGoogleCloud(t *testing.T) { token.AccessToken, ) uri += fmt.Sprintf("&%s=%s", cloud.AuthParam, cloud.AuthParamSpecified) - cloudtestutils.CheckExportStore( - t, - uri, - false, - user, - nil, /* db */ - testSettings) + info := cloudtestutils.StoreInfo{ + URI: uri, + User: user, + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", bucket, "backup-test-specified", @@ -181,9 +173,8 @@ func TestGCSAssumeRole(t *testing.T) { testSettings, ) - cloudtestutils.CheckExportStore( - t, - fmt.Sprintf("gs://%s/%s-%d?%s=%s&%s=%s&%s=%s", + info := cloudtestutils.StoreInfo{ + URI: fmt.Sprintf("gs://%s/%s-%d?%s=%s&%s=%s&%s=%s", limitedBucket, "backup-test-assume-role", testID, @@ -192,10 +183,10 @@ func TestGCSAssumeRole(t *testing.T) { AssumeRoleParam, assumedAccount, CredentialsParam, url.QueryEscape(encoded), - ), false, user, - nil, /* db */ - testSettings, - ) + ), + User: user, + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s&%s=%s", limitedBucket, "backup-test-assume-role", @@ -226,11 +217,14 @@ func TestGCSAssumeRole(t *testing.T) { testSettings, ) - cloudtestutils.CheckExportStore(t, fmt.Sprintf("gs://%s/%s-%d?%s=%s&%s=%s", limitedBucket, "backup-test-assume-role", testID, - cloud.AuthParam, cloud.AuthParamImplicit, AssumeRoleParam, assumedAccount), false, user, - nil, /* db */ - testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: fmt.Sprintf( + "gs://%s/%s-%d?%s=%s&%s=%s", limitedBucket, "backup-test-assume-role", testID, + cloud.AuthParam, cloud.AuthParamImplicit, AssumeRoleParam, assumedAccount, + ), + User: user, + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", limitedBucket, "backup-test-assume-role", @@ -291,18 +285,18 @@ func TestGCSAssumeRole(t *testing.T) { // Finally, check that the chain of roles can be used to access the storage. q.Set(AssumeRoleParam, roleChainStr) - uri := fmt.Sprintf("gs://%s/%s-%d/%s?%s", - limitedBucket, - "backup-test-assume-role", - testID, - "listing-test", - q.Encode(), - ) - cloudtestutils.CheckExportStore(t, uri, false, user, - nil, /* db */ - testSettings, - ) - cloudtestutils.CheckListFiles(t, uri, user, + info := cloudtestutils.StoreInfo{ + URI: fmt.Sprintf("gs://%s/%s-%d/%s?%s", + limitedBucket, + "backup-test-assume-role", + testID, + "listing-test", + q.Encode(), + ), + User: user, + } + cloudtestutils.CheckExportStore(t, info) + cloudtestutils.CheckListFiles(t, info.URI, user, nil, /* db */ testSettings, ) diff --git a/pkg/cloud/httpsink/http_storage_test.go b/pkg/cloud/httpsink/http_storage_test.go index ee279ee7e415..1b467d30d6ed 100644 --- a/pkg/cloud/httpsink/http_storage_test.go +++ b/pkg/cloud/httpsink/http_storage_test.go @@ -116,10 +116,12 @@ func TestPutHttp(t *testing.T) { t.Run("singleHost", func(t *testing.T) { srv, files, cleanup := makeServer() defer cleanup() - cloudtestutils.CheckExportStore(t, srv.String(), false, user, - nil, /* db */ - testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: srv.String(), + User: user, + TestSettings: testSettings, + } + cloudtestutils.CheckExportStore(t, info) if expected, actual := 14, files(); expected != actual { t.Fatalf("expected %d files to be written to single http store, got %d", expected, actual) } @@ -136,10 +138,12 @@ func TestPutHttp(t *testing.T) { combined := *srv1 combined.Host = strings.Join([]string{srv1.Host, srv2.Host, srv3.Host}, ",") - cloudtestutils.CheckExportStore(t, combined.String(), true, user, - nil, /* db */ - testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: combined.String(), + User: user, + TestSettings: testSettings, + } + cloudtestutils.CheckExportStoreSkipSingleFile(t, info) if expected, actual := 3, files1(); expected != actual { t.Fatalf("expected %d files written to http host 1, got %d", expected, actual) } diff --git a/pkg/cloud/nodelocal/nodelocal_storage_test.go b/pkg/cloud/nodelocal/nodelocal_storage_test.go index 1da276aba1b7..995d1aeb6ef9 100644 --- a/pkg/cloud/nodelocal/nodelocal_storage_test.go +++ b/pkg/cloud/nodelocal/nodelocal_storage_test.go @@ -25,8 +25,12 @@ func TestPutLocal(t *testing.T) { testSettings.ExternalIODir = p dest := MakeLocalStorageURI(p) - cloudtestutils.CheckExportStore( - t, dest, false, username.RootUserName(), nil /* db */, testSettings) + info := cloudtestutils.StoreInfo{ + URI: dest, + User: username.RootUserName(), + ExternalIODir: p, + } + cloudtestutils.CheckExportStore(t, info) url := "nodelocal://1/listing-test/basepath" cloudtestutils.CheckListFiles( t, url, username.RootUserName(), nil /*db */, testSettings, diff --git a/pkg/cloud/userfile/file_table_storage_test.go b/pkg/cloud/userfile/file_table_storage_test.go index dc5bf62d5d3c..95eea9104f14 100644 --- a/pkg/cloud/userfile/file_table_storage_test.go +++ b/pkg/cloud/userfile/file_table_storage_test.go @@ -44,15 +44,19 @@ func TestPutUserFileTable(t *testing.T) { dest := userfile.MakeUserFileStorageURI(qualifiedTableName, filename) db := s.InternalDB().(isql.DB) - cloudtestutils.CheckExportStore(t, dest, false, username.RootUserName(), db, testSettings) + info := cloudtestutils.StoreInfo{ + URI: dest, + User: username.RootUserName(), + DB: db, + } + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFiles(t, "userfile://defaultdb.public.file_list_table/listing-test/basepath", username.RootUserName(), db, testSettings) t.Run("empty-qualified-table-name", func(t *testing.T) { - dest := userfile.MakeUserFileStorageURI("", filename) - - cloudtestutils.CheckExportStore(t, dest, false, username.RootUserName(), db, testSettings) + info.URI = userfile.MakeUserFileStorageURI("", filename) + cloudtestutils.CheckExportStore(t, info) cloudtestutils.CheckListFilesCanonical(t, "userfile:///listing-test/basepath", "userfile://defaultdb.public.userfiles_root/listing-test/basepath", username.RootUserName(), db, testSettings) From 7f8c8eb8ee3d401aa35c799450e9b1a7ba24d192 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Sat, 11 Jan 2025 10:38:30 -0800 Subject: [PATCH 040/126] cloudtestutils: refactor CheckListFiles Refactor `CheckListFiles` and `CheckListFilesCanonical` to use `StoreInfo`. Epic: none Release note: None --- pkg/cloud/amazon/s3_storage_test.go | 12 +- pkg/cloud/azure/azure_storage_test.go | 30 ++--- .../cloudtestutils/cloud_test_helpers.go | 66 +++++----- pkg/cloud/gcp/gcs_storage_test.go | 120 +++++++++--------- pkg/cloud/nodelocal/BUILD.bazel | 1 - pkg/cloud/nodelocal/nodelocal_storage_test.go | 9 +- pkg/cloud/userfile/file_table_storage_test.go | 9 +- 7 files changed, 105 insertions(+), 142 deletions(-) diff --git a/pkg/cloud/amazon/s3_storage_test.go b/pkg/cloud/amazon/s3_storage_test.go index 4e6de9191ee9..923224ac44fd 100644 --- a/pkg/cloud/amazon/s3_storage_test.go +++ b/pkg/cloud/amazon/s3_storage_test.go @@ -139,9 +139,7 @@ func TestPutS3(t *testing.T) { User: user, } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles( - t, info.URI, info.User, nil /* db */, testSettings, - ) + cloudtestutils.CheckListFiles(t, info) }) // Tests that we can put an object with server side encryption specified. @@ -232,9 +230,7 @@ func TestPutS3AssumeRole(t *testing.T) { User: user, } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles( - t, info.URI, user, nil /* db */, testSettings, - ) + cloudtestutils.CheckListFiles(t, info) }) t.Run("auth-specified", func(t *testing.T) { @@ -245,9 +241,7 @@ func TestPutS3AssumeRole(t *testing.T) { User: user, } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles( - t, info.URI, user, nil /* db */, testSettings, - ) + cloudtestutils.CheckListFiles(t, info) }) t.Run("role-chaining-external-id", func(t *testing.T) { diff --git a/pkg/cloud/azure/azure_storage_test.go b/pkg/cloud/azure/azure_storage_test.go index 465815fe1431..124149b0bee0 100644 --- a/pkg/cloud/azure/azure_storage_test.go +++ b/pkg/cloud/azure/azure_storage_test.go @@ -104,7 +104,6 @@ func TestAzure(t *testing.T) { skip.IgnoreLint(t, "Test not configured for Azure") return } - testSettings := cluster.MakeTestingClusterSettings() testID := cloudtestutils.NewTestID() testPath := fmt.Sprintf("backup-test-%d", testID) testListPath := fmt.Sprintf("listing-test-%d", testID) @@ -114,29 +113,20 @@ func TestAzure(t *testing.T) { User: username.RootUserName(), } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, cfg.filePath(testListPath), - username.RootUserName(), - nil, /* db */ - testSettings, - ) + info.URI = cfg.filePath(testListPath) + cloudtestutils.CheckListFiles(t, info) // Client Secret auth info.URI = cfg.filePathClientAuth(testPath) cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, cfg.filePathClientAuth(testListPath), - username.RootUserName(), - nil, /* db */ - testSettings, - ) + info.URI = cfg.filePathClientAuth(testListPath) + cloudtestutils.CheckListFiles(t, info) // Implicit auth info.URI = cfg.filePathImplicitAuth(testPath) cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, cfg.filePathImplicitAuth(testListPath), - username.RootUserName(), - nil, /* db */ - testSettings, - ) + info.URI = cfg.filePathImplicitAuth(testListPath) + cloudtestutils.CheckListFiles(t, info) } func TestAzureSchemes(t *testing.T) { @@ -298,11 +288,7 @@ func TestAzureStorageFileImplicitAuth(t *testing.T) { URI: cfg.filePathImplicitAuth(testPath), User: username.RootUserName(), } - cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, cfg.filePathImplicitAuth(testListPath), - username.RootUserName(), - nil, /* db */ - testSettings, - ) + info.URI = cfg.filePathImplicitAuth(testListPath) + cloudtestutils.CheckListFiles(t, info) } diff --git a/pkg/cloud/cloudtestutils/cloud_test_helpers.go b/pkg/cloud/cloudtestutils/cloud_test_helpers.go index b8bf85b64dae..eaf9b70ce336 100644 --- a/pkg/cloud/cloudtestutils/cloud_test_helpers.go +++ b/pkg/cloud/cloudtestutils/cloud_test_helpers.go @@ -129,6 +129,13 @@ type StoreInfo struct { ExternalIODir string } +func (info StoreInfo) testSettings() *cluster.Settings { + if info.TestSettings != nil { + return info.TestSettings + } + return cluster.MakeTestingClusterSettings() +} + // CheckExportStore runs an array of tests against a store. func CheckExportStore(t *testing.T, info StoreInfo) { checkExportStore(t, info, false /* skipSingleFile */) @@ -144,10 +151,7 @@ func CheckExportStoreSkipSingleFile(t *testing.T, info StoreInfo) { func checkExportStore(t *testing.T, info StoreInfo, skipSingleFile bool) { ioConf := base.ExternalIODirConfig{} ctx := context.Background() - - if info.TestSettings == nil { - info.TestSettings = cluster.MakeTestingClusterSettings() - } + testSettings := info.testSettings() conf, err := cloud.ExternalStorageConfFromURI(info.URI, info.User) if err != nil { @@ -156,7 +160,7 @@ func checkExportStore(t *testing.T, info StoreInfo, skipSingleFile bool) { // Setup a sink for the given args. clientFactory := blobs.TestBlobServiceClient(info.ExternalIODir) - s, err := cloud.MakeExternalStorage(ctx, conf, ioConf, info.TestSettings, clientFactory, info.DB, nil, cloud.NilMetrics) + s, err := cloud.MakeExternalStorage(ctx, conf, ioConf, testSettings, clientFactory, info.DB, nil, cloud.NilMetrics) if err != nil { t.Fatal(err) } @@ -268,7 +272,7 @@ func checkExportStore(t *testing.T, info StoreInfo, skipSingleFile bool) { t.Fatal(err) } singleFile := storeFromURI(ctx, t, appendPath(t, info.URI, testingFilename), clientFactory, - info.User, info.DB, info.TestSettings) + info.User, info.DB, testSettings) defer singleFile.Close() res, _, err := singleFile.ReadFile(ctx, "", cloud.ReadOptions{NoFileSize: true}) @@ -289,7 +293,7 @@ func checkExportStore(t *testing.T, info StoreInfo, skipSingleFile bool) { t.Run("write-single-file-by-uri", func(t *testing.T) { const testingFilename = "B" singleFile := storeFromURI(ctx, t, appendPath(t, info.URI, testingFilename), clientFactory, - info.User, info.DB, info.TestSettings) + info.User, info.DB, testSettings) defer singleFile.Close() if err := cloud.WriteFile(ctx, singleFile, "", bytes.NewReader([]byte("bbb"))); err != nil { @@ -320,7 +324,7 @@ func checkExportStore(t *testing.T, info StoreInfo, skipSingleFile bool) { if err := cloud.WriteFile(ctx, s, testingFilename, bytes.NewReader([]byte("aaa"))); err != nil { t.Fatal(err) } - singleFile := storeFromURI(ctx, t, info.URI, clientFactory, info.User, info.DB, info.TestSettings) + singleFile := storeFromURI(ctx, t, info.URI, clientFactory, info.User, info.DB, testSettings) defer singleFile.Close() // Read a valid file. @@ -353,28 +357,16 @@ func checkExportStore(t *testing.T, info StoreInfo, skipSingleFile bool) { // CheckListFiles tests the ListFiles() interface method for the ExternalStorage // specified by storeURI. -func CheckListFiles( - t *testing.T, - storeURI string, - user username.SQLUsername, - db isql.DB, - testSettings *cluster.Settings, -) { - CheckListFilesCanonical(t, storeURI, "", user, db, testSettings) +func CheckListFiles(t *testing.T, info StoreInfo) { + CheckListFilesCanonical(t, info, "" /* canonical */) } // CheckListFilesCanonical is like CheckListFiles but takes a canonical prefix // that it should expect to see on returned listings, instead of storeURI (e.g. // if storeURI automatically expands). -func CheckListFilesCanonical( - t *testing.T, - storeURI string, - canonical string, - user username.SQLUsername, - db isql.DB, - testSettings *cluster.Settings, -) { +func CheckListFilesCanonical(t *testing.T, info StoreInfo, canonical string) { ctx := context.Background() + testSettings := info.testSettings() dataLetterFiles := []string{"file/letters/dataA.csv", "file/letters/dataB.csv", "file/letters/dataC.csv"} dataNumberFiles := []string{"file/numbers/data1.csv", "file/numbers/data2.csv", "file/numbers/data3.csv"} letterFiles := []string{"file/abc/A.csv", "file/abc/B.csv", "file/abc/C.csv"} @@ -382,9 +374,9 @@ func CheckListFilesCanonical( fileNames = append(fileNames, letterFiles...) sort.Strings(fileNames) - clientFactory := blobs.TestBlobServiceClient(testSettings.ExternalIODir) + clientFactory := blobs.TestBlobServiceClient(info.ExternalIODir) for _, fileName := range fileNames { - file := storeFromURI(ctx, t, storeURI, clientFactory, user, db, testSettings) + file := storeFromURI(ctx, t, info.URI, clientFactory, info.User, info.DB, testSettings) if err := cloud.WriteFile(ctx, file, fileName, bytes.NewReader([]byte("bbb"))); err != nil { t.Fatal(err) } @@ -409,70 +401,70 @@ func CheckListFilesCanonical( }{ { "root", - storeURI, + info.URI, "", "", foreach(fileNames, func(s string) string { return "/" + s }), }, { "file-slash-numbers-slash", - storeURI, + info.URI, "file/numbers/", "", []string{"data1.csv", "data2.csv", "data3.csv"}, }, { "root-slash", - storeURI, + info.URI, "/", "", foreach(fileNames, func(s string) string { return s }), }, { "file", - storeURI, + info.URI, "file", "", foreach(fileNames, func(s string) string { return strings.TrimPrefix(s, "file") }), }, { "file-slash", - storeURI, + info.URI, "file/", "", foreach(fileNames, func(s string) string { return strings.TrimPrefix(s, "file/") }), }, { "slash-f", - storeURI, + info.URI, "/f", "", foreach(fileNames, func(s string) string { return strings.TrimPrefix(s, "f") }), }, { "nothing", - storeURI, + info.URI, "nothing", "", nil, }, { "delim-slash-file-slash", - storeURI, + info.URI, "file/", "/", []string{"abc/", "letters/", "numbers/"}, }, { "delim-data", - storeURI, + info.URI, "", "data", []string{"/file/abc/A.csv", "/file/abc/B.csv", "/file/abc/C.csv", "/file/letters/data", "/file/numbers/data"}, }, } { t.Run(tc.name, func(t *testing.T) { - s := storeFromURI(ctx, t, tc.uri, clientFactory, user, db, testSettings) + s := storeFromURI(ctx, t, tc.uri, clientFactory, info.User, info.DB, testSettings) var actual []string require.NoError(t, s.List(ctx, tc.prefix, tc.delimiter, func(f string) error { actual = append(actual, f) @@ -485,7 +477,7 @@ func CheckListFilesCanonical( }) for _, fileName := range fileNames { - file := storeFromURI(ctx, t, storeURI, clientFactory, user, db, testSettings) + file := storeFromURI(ctx, t, info.URI, clientFactory, info.User, info.DB, testSettings) if err := file.Delete(ctx, fileName); err != nil { t.Fatal(err) } diff --git a/pkg/cloud/gcp/gcs_storage_test.go b/pkg/cloud/gcp/gcs_storage_test.go index aba83e360a55..ba2dbfd01d80 100644 --- a/pkg/cloud/gcp/gcs_storage_test.go +++ b/pkg/cloud/gcp/gcs_storage_test.go @@ -39,7 +39,6 @@ func TestPutGoogleCloud(t *testing.T) { } user := username.RootUserName() - testSettings := cluster.MakeTestingClusterSettings() testID := cloudtestutils.NewTestID() testutils.RunTrueAndFalse(t, "auth-specified-with-auth-param", func(t *testing.T, specified bool) { @@ -63,19 +62,20 @@ func TestPutGoogleCloud(t *testing.T) { User: user, } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", - bucket, - "backup-test-specified", - testID, - "listing-test", - cloud.AuthParam, - cloud.AuthParamSpecified, - CredentialsParam, - url.QueryEscape(encoded), - ), username.RootUserName(), - nil, /* db */ - testSettings, - ) + info = cloudtestutils.StoreInfo{ + URI: fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", + bucket, + "backup-test-specified", + testID, + "listing-test", + cloud.AuthParam, + cloud.AuthParamSpecified, + CredentialsParam, + url.QueryEscape(encoded), + ), + User: username.RootUserName(), + } + cloudtestutils.CheckListFiles(t, info) }) t.Run("auth-implicit", func(t *testing.T) { if !cloudtestutils.IsImplicitAuthConfigured() { @@ -87,17 +87,18 @@ func TestPutGoogleCloud(t *testing.T) { User: user, } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s", - bucket, - "backup-test-implicit", - testID, - "listing-test", - cloud.AuthParam, - cloud.AuthParamImplicit, - ), username.RootUserName(), - nil, /* db */ - testSettings, - ) + info = cloudtestutils.StoreInfo{ + URI: fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s", + bucket, + "backup-test-implicit", + testID, + "listing-test", + cloud.AuthParam, + cloud.AuthParamImplicit, + ), + User: username.RootUserName(), + } + cloudtestutils.CheckListFiles(t, info) }) t.Run("auth-specified-bearer-token", func(t *testing.T) { @@ -127,19 +128,20 @@ func TestPutGoogleCloud(t *testing.T) { User: user, } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", - bucket, - "backup-test-specified", - testID, - "listing-test", - cloud.AuthParam, - cloud.AuthParamSpecified, - BearerTokenParam, - token.AccessToken, - ), username.RootUserName(), - nil, /* db */ - testSettings, - ) + info = cloudtestutils.StoreInfo{ + URI: fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", + bucket, + "backup-test-specified", + testID, + "listing-test", + cloud.AuthParam, + cloud.AuthParamSpecified, + BearerTokenParam, + token.AccessToken, + ), + User: username.RootUserName(), + } + cloudtestutils.CheckListFiles(t, info) }) } @@ -187,21 +189,22 @@ func TestGCSAssumeRole(t *testing.T) { User: user, } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s&%s=%s", - limitedBucket, - "backup-test-assume-role", - testID, - "listing-test", - cloud.AuthParam, - cloud.AuthParamSpecified, - AssumeRoleParam, - assumedAccount, - CredentialsParam, - url.QueryEscape(encoded), - ), username.RootUserName(), - nil, /* db */ - testSettings, - ) + info = cloudtestutils.StoreInfo{ + URI: fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s&%s=%s", + limitedBucket, + "backup-test-assume-role", + testID, + "listing-test", + cloud.AuthParam, + cloud.AuthParamSpecified, + AssumeRoleParam, + assumedAccount, + CredentialsParam, + url.QueryEscape(encoded), + ), + User: username.RootUserName(), + } + cloudtestutils.CheckListFiles(t, info) }) t.Run("implicit", func(t *testing.T) { @@ -225,7 +228,7 @@ func TestGCSAssumeRole(t *testing.T) { User: user, } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", + info.URI = fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", limitedBucket, "backup-test-assume-role", testID, @@ -234,10 +237,8 @@ func TestGCSAssumeRole(t *testing.T) { cloud.AuthParamImplicit, AssumeRoleParam, assumedAccount, - ), username.RootUserName(), - nil, /* db */ - testSettings, ) + cloudtestutils.CheckListFiles(t, info) }) t.Run("role-chaining", func(t *testing.T) { @@ -296,10 +297,7 @@ func TestGCSAssumeRole(t *testing.T) { User: user, } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, info.URI, user, - nil, /* db */ - testSettings, - ) + cloudtestutils.CheckListFiles(t, info) }) } }) diff --git a/pkg/cloud/nodelocal/BUILD.bazel b/pkg/cloud/nodelocal/BUILD.bazel index c7505edc039f..11105cf01b6d 100644 --- a/pkg/cloud/nodelocal/BUILD.bazel +++ b/pkg/cloud/nodelocal/BUILD.bazel @@ -36,7 +36,6 @@ go_test( deps = [ "//pkg/cloud/cloudtestutils", "//pkg/security/username", - "//pkg/settings/cluster", "//pkg/testutils", "//pkg/util/leaktest", ], diff --git a/pkg/cloud/nodelocal/nodelocal_storage_test.go b/pkg/cloud/nodelocal/nodelocal_storage_test.go index 995d1aeb6ef9..23209f48155f 100644 --- a/pkg/cloud/nodelocal/nodelocal_storage_test.go +++ b/pkg/cloud/nodelocal/nodelocal_storage_test.go @@ -10,7 +10,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud/cloudtestutils" "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -21,8 +20,6 @@ func TestPutLocal(t *testing.T) { p, cleanupFn := testutils.TempDir(t) defer cleanupFn() - testSettings := cluster.MakeTestingClusterSettings() - testSettings.ExternalIODir = p dest := MakeLocalStorageURI(p) info := cloudtestutils.StoreInfo{ @@ -31,8 +28,6 @@ func TestPutLocal(t *testing.T) { ExternalIODir: p, } cloudtestutils.CheckExportStore(t, info) - url := "nodelocal://1/listing-test/basepath" - cloudtestutils.CheckListFiles( - t, url, username.RootUserName(), nil /*db */, testSettings, - ) + info.URI = "nodelocal://1/listing-test/basepath" + cloudtestutils.CheckListFiles(t, info) } diff --git a/pkg/cloud/userfile/file_table_storage_test.go b/pkg/cloud/userfile/file_table_storage_test.go index 95eea9104f14..97fbdaa6a9ed 100644 --- a/pkg/cloud/userfile/file_table_storage_test.go +++ b/pkg/cloud/userfile/file_table_storage_test.go @@ -39,7 +39,6 @@ func TestPutUserFileTable(t *testing.T) { srv := serverutils.StartServerOnly(t, base.TestServerArgs{}) defer srv.Stopper().Stop(ctx) s := srv.ApplicationLayer() - testSettings := s.ClusterSettings() dest := userfile.MakeUserFileStorageURI(qualifiedTableName, filename) @@ -51,15 +50,15 @@ func TestPutUserFileTable(t *testing.T) { } cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFiles(t, "userfile://defaultdb.public.file_list_table/listing-test/basepath", - username.RootUserName(), db, testSettings) + info.URI = "userfile://defaultdb.public.file_list_table/listing-test/basepath" + cloudtestutils.CheckListFiles(t, info) t.Run("empty-qualified-table-name", func(t *testing.T) { info.URI = userfile.MakeUserFileStorageURI("", filename) cloudtestutils.CheckExportStore(t, info) - cloudtestutils.CheckListFilesCanonical(t, "userfile:///listing-test/basepath", "userfile://defaultdb.public.userfiles_root/listing-test/basepath", - username.RootUserName(), db, testSettings) + info.URI = "userfile:///listing-test/basepath" + cloudtestutils.CheckListFilesCanonical(t, info, "userfile://defaultdb.public.userfiles_root/listing-test/basepath") }) t.Run("reject-normalized-basename", func(t *testing.T) { From f76f279423152185dedcd784bd9881f281a651c9 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Sat, 11 Jan 2025 11:45:02 -0800 Subject: [PATCH 041/126] cloudtestutils: refactor CheckNoPermissions Use `StoreInfo` for `CheckNoPermissions.` Epic: none Release note: None --- pkg/cloud/amazon/s3_storage_test.go | 30 ++++---- pkg/cloud/azure/azure_storage_test.go | 12 ++- .../cloudtestutils/cloud_test_helpers.go | 17 ++--- pkg/cloud/gcp/gcs_storage_test.go | 74 +++++++++---------- 4 files changed, 59 insertions(+), 74 deletions(-) diff --git a/pkg/cloud/amazon/s3_storage_test.go b/pkg/cloud/amazon/s3_storage_test.go index 923224ac44fd..a0a9cefab9b7 100644 --- a/pkg/cloud/amazon/s3_storage_test.go +++ b/pkg/cloud/amazon/s3_storage_test.go @@ -210,7 +210,6 @@ func TestPutS3AssumeRole(t *testing.T) { skip.IgnoreLint(t, "AWS_S3_BUCKET env var must be set") } - testSettings := cluster.MakeTestingClusterSettings() testID := cloudtestutils.NewTestID() testPath := fmt.Sprintf("backup-test-%d", testID) @@ -263,18 +262,19 @@ func TestPutS3AssumeRole(t *testing.T) { t.Run(tc.auth, func(t *testing.T) { // First verify that none of the individual roles in the chain can be used to access the storage. for _, p := range providerChain { - roleURI := S3URI(bucket, testPath, - &cloudpb.ExternalStorage_S3{ - Auth: tc.auth, - AssumeRoleProvider: p, - AccessKey: tc.accessKey, - Secret: tc.secretKey, - Region: "us-east-1", - }, - ) - cloudtestutils.CheckNoPermission( - t, roleURI, user, nil /* db */, testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: S3URI(bucket, testPath, + &cloudpb.ExternalStorage_S3{ + Auth: tc.auth, + AssumeRoleProvider: p, + AccessKey: tc.accessKey, + Secret: tc.secretKey, + Region: "us-east-1", + }, + ), + User: user, + } + cloudtestutils.CheckNoPermission(t, info) } // Next check that the role chain without any external IDs cannot be used to @@ -298,9 +298,7 @@ func TestPutS3AssumeRole(t *testing.T) { ), User: user, } - cloudtestutils.CheckNoPermission( - t, info.URI, user, nil /* db */, testSettings, - ) + cloudtestutils.CheckNoPermission(t, info) // Finally, check that the chain of roles can be used to access the storage. info.URI = S3URI(bucket, testPath, diff --git a/pkg/cloud/azure/azure_storage_test.go b/pkg/cloud/azure/azure_storage_test.go index 124149b0bee0..9bbf1e597865 100644 --- a/pkg/cloud/azure/azure_storage_test.go +++ b/pkg/cloud/azure/azure_storage_test.go @@ -263,7 +263,6 @@ func TestAzureStorageFileImplicitAuth(t *testing.T) { skip.IgnoreLint(t, "Test not configured for Azure") return } - testSettings := cluster.MakeTestingClusterSettings() testID := cloudtestutils.NewTestID() cleanup := envutil.TestSetEnv(t, "AZURE_CLIENT_ID", "") @@ -272,8 +271,11 @@ func TestAzureStorageFileImplicitAuth(t *testing.T) { testPath := fmt.Sprintf("backup-test-%d", testID) testListPath := fmt.Sprintf("listing-test-%d", testID) - cloudtestutils.CheckNoPermission(t, cfg.filePathImplicitAuth(testPath), username.RootUserName(), - nil /*db*/, testSettings) + info := cloudtestutils.StoreInfo{ + URI: cfg.filePathImplicitAuth(testPath), + User: username.RootUserName(), + } + cloudtestutils.CheckNoPermission(t, info) tmpDir, cleanup2 := testutils.TempDir(t) defer cleanup2() @@ -284,10 +286,6 @@ func TestAzureStorageFileImplicitAuth(t *testing.T) { cleanup3 := envutil.TestSetEnv(t, "COCKROACH_AZURE_APPLICATION_CREDENTIALS_FILE", credFile) defer cleanup3() - info := cloudtestutils.StoreInfo{ - URI: cfg.filePathImplicitAuth(testPath), - User: username.RootUserName(), - } cloudtestutils.CheckExportStore(t, info) info.URI = cfg.filePathImplicitAuth(testListPath) cloudtestutils.CheckListFiles(t, info) diff --git a/pkg/cloud/cloudtestutils/cloud_test_helpers.go b/pkg/cloud/cloudtestutils/cloud_test_helpers.go index eaf9b70ce336..5cf1a7e8be80 100644 --- a/pkg/cloud/cloudtestutils/cloud_test_helpers.go +++ b/pkg/cloud/cloudtestutils/cloud_test_helpers.go @@ -558,24 +558,19 @@ func CheckAntagonisticRead( // CheckNoPermission checks that we do not have permission to list the external // storage at storeURI. -func CheckNoPermission( - t *testing.T, - storeURI string, - user username.SQLUsername, - db isql.DB, - testSettings *cluster.Settings, -) { +func CheckNoPermission(t *testing.T, info StoreInfo) { ioConf := base.ExternalIODirConfig{} ctx := context.Background() - conf, err := cloud.ExternalStorageConfFromURI(storeURI, user) + conf, err := cloud.ExternalStorageConfFromURI(info.URI, info.User) if err != nil { t.Fatal(err) } - clientFactory := blobs.TestBlobServiceClient(testSettings.ExternalIODir) + testSettings := info.testSettings() + clientFactory := blobs.TestBlobServiceClient(info.ExternalIODir) s, err := cloud.MakeExternalStorage( - ctx, conf, ioConf, testSettings, clientFactory, db, nil, cloud.NilMetrics, + ctx, conf, ioConf, testSettings, clientFactory, info.DB, nil, cloud.NilMetrics, ) if err != nil { t.Fatal(err) @@ -584,7 +579,7 @@ func CheckNoPermission( err = s.List(ctx, "", "", nil) if err == nil { - t.Fatalf("expected error when listing %s with no permissions", storeURI) + t.Fatalf("expected error when listing %s with no permissions", info.URI) } require.Regexp(t, "(failed|unable) to list", err) diff --git a/pkg/cloud/gcp/gcs_storage_test.go b/pkg/cloud/gcp/gcs_storage_test.go index ba2dbfd01d80..4ab1d2f8b865 100644 --- a/pkg/cloud/gcp/gcs_storage_test.go +++ b/pkg/cloud/gcp/gcs_storage_test.go @@ -147,7 +147,6 @@ func TestPutGoogleCloud(t *testing.T) { func TestGCSAssumeRole(t *testing.T) { user := username.RootUserName() - testSettings := cluster.MakeTestingClusterSettings() limitedBucket := os.Getenv("GOOGLE_LIMITED_BUCKET") if limitedBucket == "" { @@ -167,27 +166,25 @@ func TestGCSAssumeRole(t *testing.T) { } encoded := base64.StdEncoding.EncodeToString([]byte(credentials)) - // Verify that specified permissions with the credentials do not give us - // access to the bucket. - cloudtestutils.CheckNoPermission(t, fmt.Sprintf("gs://%s/%s-%d?%s=%s", limitedBucket, "backup-test-assume-role", testID, - CredentialsParam, url.QueryEscape(encoded)), user, - nil, /* db */ - testSettings, - ) - info := cloudtestutils.StoreInfo{ - URI: fmt.Sprintf("gs://%s/%s-%d?%s=%s&%s=%s&%s=%s", - limitedBucket, - "backup-test-assume-role", - testID, - cloud.AuthParam, - cloud.AuthParamSpecified, - AssumeRoleParam, - assumedAccount, CredentialsParam, - url.QueryEscape(encoded), - ), + URI: fmt.Sprintf( + "gs://%s/%s-%d?%s=%s", limitedBucket, "backup-test-assume-role", testID, + CredentialsParam, url.QueryEscape(encoded)), User: user, } + // Verify that specified permissions with the credentials do not give us + // access to the bucket. + cloudtestutils.CheckNoPermission(t, info) + info.URI = fmt.Sprintf("gs://%s/%s-%d?%s=%s&%s=%s&%s=%s", + limitedBucket, + "backup-test-assume-role", + testID, + cloud.AuthParam, + cloud.AuthParamSpecified, + AssumeRoleParam, + assumedAccount, CredentialsParam, + url.QueryEscape(encoded), + ) cloudtestutils.CheckExportStore(t, info) info = cloudtestutils.StoreInfo{ URI: fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s&%s=%s", @@ -212,21 +209,18 @@ func TestGCSAssumeRole(t *testing.T) { skip.IgnoreLint(t, err) } - // Verify that implicit permissions with the credentials do not give us - // access to the bucket. - cloudtestutils.CheckNoPermission(t, fmt.Sprintf("gs://%s/%s-%d?%s=%s", limitedBucket, "backup-test-assume-role", testID, - cloud.AuthParam, cloud.AuthParamImplicit), user, - nil, /* db */ - testSettings, - ) - info := cloudtestutils.StoreInfo{ URI: fmt.Sprintf( - "gs://%s/%s-%d?%s=%s&%s=%s", limitedBucket, "backup-test-assume-role", testID, - cloud.AuthParam, cloud.AuthParamImplicit, AssumeRoleParam, assumedAccount, + "gs://%s/%s-%d?%s=%s", limitedBucket, "backup-test-assume-role", testID, + cloud.AuthParam, cloud.AuthParamImplicit, ), User: user, } + cloudtestutils.CheckNoPermission(t, info) + info.URI = fmt.Sprintf( + "gs://%s/%s-%d?%s=%s&%s=%s", limitedBucket, "backup-test-assume-role", testID, + cloud.AuthParam, cloud.AuthParamImplicit, AssumeRoleParam, assumedAccount, + ) cloudtestutils.CheckExportStore(t, info) info.URI = fmt.Sprintf("gs://%s/%s-%d/%s?%s=%s&%s=%s", limitedBucket, @@ -271,17 +265,17 @@ func TestGCSAssumeRole(t *testing.T) { // to access the storage. for _, role := range roleChain { q.Set(AssumeRoleParam, role) - roleURI := fmt.Sprintf("gs://%s/%s-%d/%s?%s", - limitedBucket, - "backup-test-assume-role", - testID, - "listing-test", - q.Encode(), - ) - cloudtestutils.CheckNoPermission(t, roleURI, user, - nil, /* db */ - testSettings, - ) + info := cloudtestutils.StoreInfo{ + URI: fmt.Sprintf("gs://%s/%s-%d/%s?%s", + limitedBucket, + "backup-test-assume-role", + testID, + "listing-test", + q.Encode(), + ), + User: user, + } + cloudtestutils.CheckNoPermission(t, info) } // Finally, check that the chain of roles can be used to access the storage. From 7a066c2d5f9aa28384ec6c53ed72610131773732 Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Fri, 10 Jan 2025 18:53:58 -0500 Subject: [PATCH 042/126] admission: account for large flush backlog WAL failover permits a large flush backlog. This causes the accounted writes in admission control to be significantly larger than the bytes flushed to L0 (when the backlog is building up) and be significantly smaller when the backlog is clearing up (or when a huge flush completes, even if the backlog is stable). This causes the write linear model to be very wrong. Additionally, we can give out unlimited elastic tokens if the current overload score is very low. This PR fixes this behavior by doing the following when the backlog is high or clearing up: - Not adjusting the write linear model or the above-raft model (only used rarely with RACv2). - Upper bounding the elastic tokens by the compaction bytes out of L0. The changes here slow down the rate at which send-queues are emptied in perturbation/full/partition when the partition is removed, and there is no unnecessary queueing of regular work in the AC WorkQueues. For more details see https://github.com/cockroachdb/cockroach/issues/138798#issuecomment-2585885553. Fixes #138798 Epic: none Release note: None --- pkg/server/node.go | 14 +- pkg/storage/engine.go | 4 + pkg/storage/pebble.go | 5 + pkg/util/admission/granter.go | 4 +- pkg/util/admission/io_load_listener.go | 80 +++++++--- pkg/util/admission/io_load_listener_test.go | 8 +- pkg/util/admission/store_token_estimation.go | 9 +- .../admission/store_token_estimation_test.go | 7 +- pkg/util/admission/testdata/io_load_listener | 145 +++++++++++------- .../testdata/store_per_work_token_estimator | 21 +++ 10 files changed, 211 insertions(+), 86 deletions(-) diff --git a/pkg/server/node.go b/pkg/server/node.go index 34d9759d72dd..91b8c75ae442 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -1377,16 +1377,20 @@ func (pmp *nodePebbleMetricsProvider) GetPebbleMetrics() []admission.StoreMetric } var metrics []admission.StoreMetrics _ = pmp.n.stores.VisitStores(func(store *kvserver.Store) error { - m := store.TODOEngine().GetMetrics() + eng := store.TODOEngine() + m := eng.GetMetrics() + opts := eng.GetPebbleOptions() + memTableSizeForStopWrites := opts.MemTableSize * uint64(opts.MemTableStopWritesThreshold) diskStats := admission.DiskStats{ProvisionedBandwidth: clusterProvisionedBandwidth} if s, ok := storeIDToDiskStats[store.StoreID()]; ok { diskStats = s } metrics = append(metrics, admission.StoreMetrics{ - StoreID: store.StoreID(), - Metrics: m.Metrics, - WriteStallCount: m.WriteStallCount, - DiskStats: diskStats, + StoreID: store.StoreID(), + Metrics: m.Metrics, + WriteStallCount: m.WriteStallCount, + DiskStats: diskStats, + MemTableSizeForStopWrites: memTableSizeForStopWrites, }) return nil }) diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index d4bc877451a7..b0c64268e0f6 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -1129,6 +1129,10 @@ type Engine interface { // needs to be thread-safe as it could be called repeatedly in multiple threads // over a short period of time. RegisterDiskSlowCallback(cb func(info pebble.DiskSlowInfo)) + + // GetPebbleOptions returns the options used when creating the engine. The + // caller must not modify these. + GetPebbleOptions() *pebble.Options } // Batch is the interface for batch specific operations. diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 227536672aa3..11510a73777c 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -2137,6 +2137,11 @@ func (p *Pebble) GetMetrics() Metrics { return m } +// GetPebbleOptions implements the Engine interface. +func (p *Pebble) GetPebbleOptions() *pebble.Options { + return p.cfg.opts +} + // GetEncryptionRegistries implements the Engine interface. func (p *Pebble) GetEncryptionRegistries() (*fs.EncryptionRegistries, error) { rv := &fs.EncryptionRegistries{} diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 166dce3af7b9..74e6aaa4e23a 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -799,13 +799,15 @@ type IOThresholdConsumer interface { UpdateIOThreshold(roachpb.StoreID, *admissionpb.IOThreshold) } -// StoreMetrics are the metrics for a store. +// StoreMetrics are the metrics and some config information for a store. type StoreMetrics struct { StoreID roachpb.StoreID *pebble.Metrics WriteStallCount int64 // Optional. DiskStats DiskStats + // Config. + MemTableSizeForStopWrites uint64 } // DiskStats provide low-level stats about the disk resources used for a diff --git a/pkg/util/admission/io_load_listener.go b/pkg/util/admission/io_load_listener.go index 231ae65a1660..5afa4f06b726 100644 --- a/pkg/util/admission/io_load_listener.go +++ b/pkg/util/admission/io_load_listener.go @@ -113,6 +113,10 @@ var L0MinimumSizePerSubLevel = settings.RegisterIntSetting( "when non-zero, this indicates the minimum size that is needed to count towards one sub-level", 5<<20, settings.NonNegativeInt) +// This flag was introduced before any experience with WAL failover. It turns +// out that WAL failover can sometimes be triggered because the disk is +// overloaded, and allowing unlimited tokens is going to make it worse. So do +// not set this to true without consulting with an expert. var walFailoverUnlimitedTokens = settings.RegisterBoolSetting( settings.SystemOnly, "admission.wal.failover.unlimited_tokens.enabled", @@ -250,6 +254,7 @@ type ioLoadListenerState struct { } cumCompactionStats cumStoreCompactionStats cumWALSecondaryWriteDuration time.Duration + unflushedMemTableTooLarge bool // Exponentially smoothed per interval values. @@ -541,7 +546,7 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMe sas := io.kvRequester.getStoreAdmissionStats() cumIngestBytes := cumLSMIngestedBytes(metrics.Metrics) io.perWorkTokenEstimator.updateEstimates( - metrics.Levels[0], cumIngestBytes, metrics.DiskStats.BytesWritten, sas) + metrics.Levels[0], cumIngestBytes, metrics.DiskStats.BytesWritten, sas, false) io.adjustTokensResult = adjustTokensResult{ ioLoadListenerState: ioLoadListenerState{ cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested, @@ -725,6 +730,8 @@ func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics L0SubLevelCountOverloadThreshold.Get(&io.settings.SV), L0MinimumSizePerSubLevel.Get(&io.settings.SV), MinFlushUtilizationFraction.Get(&io.settings.SV), + metrics.MemTable.Size, + metrics.MemTableSizeForStopWrites, ) io.adjustTokensResult = res cumIngestedBytes := cumLSMIngestedBytes(metrics.Metrics) @@ -753,7 +760,9 @@ func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics io.diskBW.bytesRead = metrics.DiskStats.BytesRead io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten - io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumIngestedBytes, metrics.DiskStats.BytesWritten, sas) + io.perWorkTokenEstimator.updateEstimates( + metrics.Levels[0], cumIngestedBytes, metrics.DiskStats.BytesWritten, sas, + io.aux.recentUnflushedMemTableTooLarge) io.copyAuxEtcFromPerWorkEstimator() requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission() io.kvRequester.setStoreRequestEstimates(requestEstimates) @@ -799,10 +808,11 @@ type adjustTokensAuxComputations struct { intWALFailover bool - prevTokensUsed int64 - prevTokensUsedByElasticWork int64 - tokenKind tokenKind - usedCompactionTokensLowerBound bool + prevTokensUsed int64 + prevTokensUsedByElasticWork int64 + tokenKind tokenKind + usedCompactionTokensLowerBound bool + recentUnflushedMemTableTooLarge bool perWorkTokensAux perWorkTokensAux doLogFlush bool @@ -821,6 +831,8 @@ func (io *ioLoadListener) adjustTokensInner( threshNumFiles, threshNumSublevels int64, l0MinSizePerSubLevel int64, minFlushUtilTargetFraction float64, + memTableSize uint64, + memTableSizeForStopWrites uint64, ) adjustTokensResult { ioThreshold := &admissionpb.IOThreshold{ L0NumFiles: l0Metrics.NumFiles, @@ -830,6 +842,12 @@ func (io *ioLoadListener) adjustTokensInner( L0Size: l0Metrics.Size, L0MinimumSizePerSubLevel: l0MinSizePerSubLevel, } + unflushedMemTableTooLarge := memTableSize > memTableSizeForStopWrites + // If it was too large in the last sample 15s ago, and is not large now, the + // stats will still be skewed towards showing disproportionate L0 bytes + // added compared to incoming writes to Pebble. So we include this limited + // history. + recentUnflushedMemTableTooLarge := unflushedMemTableTooLarge || io.unflushedMemTableTooLarge curL0Bytes := l0Metrics.Size cumL0AddedBytes := l0Metrics.BytesFlushed + l0Metrics.BytesIngested @@ -1192,6 +1210,25 @@ func (io *ioLoadListener) adjustTokensInner( } totalNumElasticByteTokens = max(totalNumElasticByteTokens, 1) } + if recentUnflushedMemTableTooLarge { + // There is a large flush backlog -- this will typically happen during or + // immediately after WAL failover. Don't allow elastic work to get more + // than the bytes we compacted out of L0. There are other choices we could + // have made here (a) give 1 token to elastic work, effectively throttling + // it down to zero, (b) chosen some other fraction of intL0CompactedBytes. + // With (a), we run the risk of rapidly switching back and forth between + // giving elastic work 1 token and unlimitedTokens, which could be worse + // than staying in this mode of a flush backlog. With (b), we don't have a + // good idea of what fraction would be appropriate. Note that this is + // mainly a way to avoid overloading the disk. If disk bandwidth based + // tokens are enabled, those should be observing the cost of large ongoing + // flushes and throttling elastic work, and there should be less need of + // this mechanism. + if totalNumElasticByteTokens > intL0CompactedBytes { + doLogFlush = true + totalNumElasticByteTokens = intL0CompactedBytes + } + } // Use the minimum of the token count calculated using compactions and // flushes. tokenKind := compactionTokenKind @@ -1223,6 +1260,7 @@ func (io *ioLoadListener) adjustTokensInner( cumWriteStallCount: cumWriteStallCount, cumCompactionStats: cumCompactionStats, cumWALSecondaryWriteDuration: cumWALSecondaryWriteDuration, + unflushedMemTableTooLarge: unflushedMemTableTooLarge, smoothedIntL0CompactedBytes: smoothedIntL0CompactedBytes, smoothedCompactionByteTokens: smoothedCompactionByteTokens, smoothedNumFlushTokens: smoothedNumFlushTokens, @@ -1235,17 +1273,18 @@ func (io *ioLoadListener) adjustTokensInner( elasticByteTokensAllocated: 0, }, aux: adjustTokensAuxComputations{ - intL0AddedBytes: intL0AddedBytes, - intL0CompactedBytes: intL0CompactedBytes, - intFlushTokens: intFlushTokens, - intFlushUtilization: intFlushUtilization, - intWriteStalls: intWriteStalls, - intWALFailover: intWALFailover, - prevTokensUsed: prev.byteTokensUsed, - prevTokensUsedByElasticWork: prev.byteTokensUsedByElasticWork, - tokenKind: tokenKind, - usedCompactionTokensLowerBound: usedCompactionTokensLowerBound, - doLogFlush: doLogFlush, + intL0AddedBytes: intL0AddedBytes, + intL0CompactedBytes: intL0CompactedBytes, + intFlushTokens: intFlushTokens, + intFlushUtilization: intFlushUtilization, + intWriteStalls: intWriteStalls, + intWALFailover: intWALFailover, + prevTokensUsed: prev.byteTokensUsed, + prevTokensUsedByElasticWork: prev.byteTokensUsedByElasticWork, + tokenKind: tokenKind, + usedCompactionTokensLowerBound: usedCompactionTokensLowerBound, + recentUnflushedMemTableTooLarge: recentUnflushedMemTableTooLarge, + doLogFlush: doLogFlush, }, ioThreshold: ioThreshold, } @@ -1267,8 +1306,13 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) { ib := humanizeutil.IBytes // NB: "≈" indicates smoothed quantities. p.Printf("compaction score %v (%d ssts, %d sub-levels), ", res.ioThreshold, res.ioThreshold.L0NumFiles, res.ioThreshold.L0NumSubLevels) - p.Printf("L0 growth %s (write %s (ignored %s) ingest %s (ignored %s)): ", + var recentFlushBackogStr string + if res.aux.recentUnflushedMemTableTooLarge { + recentFlushBackogStr = " (flush-backlog) " + } + p.Printf("L0 growth %s%s (write %s (ignored %s) ingest %s (ignored %s)): ", ib(res.aux.intL0AddedBytes), + redact.SafeString(recentFlushBackogStr), ib(res.aux.perWorkTokensAux.intL0WriteBytes), ib(res.aux.perWorkTokensAux.intL0IgnoredWriteBytes), ib(res.aux.perWorkTokensAux.intL0IngestedBytes), diff --git a/pkg/util/admission/io_load_listener_test.go b/pkg/util/admission/io_load_listener_test.go index 0d4617696064..7f3623c07a24 100644 --- a/pkg/util/admission/io_load_listener_test.go +++ b/pkg/util/admission/io_load_listener_test.go @@ -214,7 +214,10 @@ func TestIOLoadListener(t *testing.T) { if d.HasArg("loaded") { currDuration = loadedDuration } - + memTableSizeForStopWrites := uint64(256 << 20) + if d.HasArg("unflushed-too-large") { + metrics.MemTable.Size = memTableSizeForStopWrites + 1 + } ioll.pebbleMetricsTick(ctx, StoreMetrics{ Metrics: &metrics, WriteStallCount: int64(writeStallCount), @@ -223,6 +226,7 @@ func TestIOLoadListener(t *testing.T) { BytesWritten: uint64(bytesWritten), ProvisionedBandwidth: int64(provisionedBandwidth), }, + MemTableSizeForStopWrites: memTableSizeForStopWrites, }) var buf strings.Builder // Do the ticks until just before next adjustment. @@ -326,7 +330,7 @@ func TestAdjustTokensInnerAndLogging(t *testing.T) { } res := ioll.adjustTokensInner( ctx, tt.prev, tt.l0Metrics, 12, cumStoreCompactionStats{numOutLevelsGauge: 1}, 0, - pebble.ThroughputMetric{}, 100, 10, 0, 0.50) + pebble.ThroughputMetric{}, 100, 10, 0, 0.50, 10, 100) buf.Printf("%s\n", res) } echotest.Require(t, string(redact.Sprint(buf)), filepath.Join(datapathutils.TestDataPath(t, "format_adjust_tokens_stats.txt"))) diff --git a/pkg/util/admission/store_token_estimation.go b/pkg/util/admission/store_token_estimation.go index 97fa94c3a2b7..7407c887a880 100644 --- a/pkg/util/admission/store_token_estimation.go +++ b/pkg/util/admission/store_token_estimation.go @@ -197,6 +197,7 @@ func (e *storePerWorkTokenEstimator) updateEstimates( cumLSMIngestedBytes uint64, cumDiskWrite uint64, admissionStats storeAdmissionStats, + unflushedMemTableTooLarge bool, ) { if e.cumL0WriteBytes == 0 { e.cumStoreAdmissionStats = admissionStats @@ -228,8 +229,10 @@ func (e *storePerWorkTokenEstimator) updateEstimates( // many did go to L0. intIngestedAccountedBytes := int64(admissionStats.ingestedAccountedBytes) - int64(e.cumStoreAdmissionStats.ingestedAccountedBytes) - e.atDoneL0WriteTokensLinearModel.updateModelUsingIntervalStats( - intL0WriteAccountedBytes, adjustedIntL0WriteBytes, intWorkCount) + if !unflushedMemTableTooLarge { + e.atDoneL0WriteTokensLinearModel.updateModelUsingIntervalStats( + intL0WriteAccountedBytes, adjustedIntL0WriteBytes, intWorkCount) + } e.atDoneL0IngestTokensLinearModel.updateModelUsingIntervalStats( intIngestedAccountedBytes, adjustedIntL0IngestedBytes, intWorkCount) // Ingest across all levels model. @@ -260,7 +263,7 @@ func (e *storePerWorkTokenEstimator) updateEstimates( int64(e.cumStoreAdmissionStats.aboveRaftStats.writeAccountedBytes) intAboveRaftIngestedAccountedBytes := int64(admissionStats.aboveRaftStats.ingestedAccountedBytes) - int64(e.cumStoreAdmissionStats.aboveRaftStats.ingestedAccountedBytes) - if intAboveRaftWorkCount > 1 && intL0TotalBytes > 0 { + if intAboveRaftWorkCount > 1 && intL0TotalBytes > 0 && !unflushedMemTableTooLarge { // We don't know how many of the intL0TotalBytes (which is a stat derived // from Pebble stats) are due to above-raft admission. So we simply apply // the linear models to the stats we have and then use the modeled bytes diff --git a/pkg/util/admission/store_token_estimation_test.go b/pkg/util/admission/store_token_estimation_test.go index 4fb828d90bc0..a81328036903 100644 --- a/pkg/util/admission/store_token_estimation_test.go +++ b/pkg/util/admission/store_token_estimation_test.go @@ -95,7 +95,12 @@ func TestStorePerWorkTokenEstimator(t *testing.T) { d.ScanArgs(t, "disk-writes", &diskWrites) cumDiskWrites += uint64(diskWrites) } - estimator.updateEstimates(l0Metrics, cumLSMIngestedBytes, cumDiskWrites, admissionStats) + unflushedTooLarge := false + if d.HasArg("unflushed-too-large") { + unflushedTooLarge = true + } + estimator.updateEstimates( + l0Metrics, cumLSMIngestedBytes, cumDiskWrites, admissionStats, unflushedTooLarge) wL0lm, iL0lm, ilm, wamplm := estimator.getModelsAtDone() require.Equal(t, wL0lm, estimator.atDoneL0WriteTokensLinearModel.smoothedLinearModel) require.Equal(t, iL0lm, estimator.atDoneL0IngestTokensLinearModel.smoothedLinearModel) diff --git a/pkg/util/admission/testdata/io_load_listener b/pkg/util/admission/testdata/io_load_listener index 7e620430d8e2..8820ddabfe35 100644 --- a/pkg/util/admission/testdata/io_load_listener +++ b/pkg/util/admission/testdata/io_load_listener @@ -13,7 +13,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=21 l0-sublevels=21 ---- compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false tick: 1, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false tick: 2, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -87,7 +87,7 @@ set-state l0-bytes=10000 l0-added-write=101000 l0-files=21 l0-sublevels=21 ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.00); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 5 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.00x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=209(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -160,7 +160,7 @@ set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21 ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 2.25x+1 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 7 B, compacted 98 KiB [≈73 KiB], flushed 1.4 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 7 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -229,7 +229,7 @@ set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21 print ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 7 B, compacted 0 B [≈37 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting 21 KiB (rate 1.4 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:21875 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:21875 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:7} l0WriteLM:{multiplier:2.125 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 7 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.12x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=365(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=365(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -244,7 +244,7 @@ set-state l0-bytes=10000 l0-added-write=501000 l0-files=21 l0-sublevels=9 print- ---- compaction score 0.450 (21 ssts, 9 sub-levels), L0 growth 293 KiB (write 293 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 293 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 3.00x+18 B (smoothed 2.56x+9 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 18 B, compacted 293 KiB [≈165 KiB], flushed 4.3 MiB [≈0 B] (mult 1.00); admitting 110 KiB (rate 7.3 KiB/s) (elastic 62 KiB rate 4.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:112187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:112187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:9} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intFlushTokens:4.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:300000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:3 constant:18} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:300000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:112187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:112187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:9} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intFlushTokens:4.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:300000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:3 constant:18} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:300000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 18 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+9 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1870(elastic 1055) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1870(elastic 1055) max-disk-bw-tokens=unlimited lastTick=false @@ -257,7 +257,7 @@ set-state l0-bytes=10000 l0-added-write=501000 l0-files=21 l0-sublevels=6 print- ---- compaction score 0.300 (21 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 2.56x+4 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 18 B, compacted 0 B [≈82 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting 129 KiB (rate 8.6 KiB/s) (elastic 62 KiB rate 4.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:84375 smoothedCompactionByteTokens:132031.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:132031 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:4} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:84375 smoothedCompactionByteTokens:132031.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:132031 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:63281 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:4} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 18 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+4 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=2201(elastic 1055) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=2201(elastic 1055) max-disk-bw-tokens=unlimited lastTick=false @@ -270,7 +270,7 @@ set-state l0-bytes=10000 l0-added-write=501000 l0-files=21 l0-sublevels=3 print- ---- compaction score 0.150 (21 ssts, 3 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 2.56x+2 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 18 B, compacted 0 B [≈41 KiB], flushed 0 B [≈0 B] (mult 1.00); admitting elastic 46 KiB (rate 3.1 KiB/s) due to L0 growth; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:7000000000 IdleDuration:700000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:42187 smoothedCompactionByteTokens:66015.625 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:47460 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:2} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:7000000000 IdleDuration:700000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:42187 smoothedCompactionByteTokens:66015.625 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:47460 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:18} l0WriteLM:{multiplier:2.5625 constant:2} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 18 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.56x+2 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic 791) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic 791) max-disk-bw-tokens=unlimited lastTick=false @@ -288,7 +288,7 @@ set-state l0-bytes=1000 l0-added-write=1000 l0-added-ingested=0 l0-files=21 l0-s ---- compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # L0 will see an addition of 200,000 bytes. 150,000 bytes were mentioned by @@ -306,7 +306,7 @@ set-state l0-bytes=1000 l0-added-write=171000 l0-added-ingested=30000 l0-files=2 ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB (write 166 KiB (ignored 0 B) ingest 29 KiB (ignored 0 B)): requests 10 (0 bypassed) with 127 KiB acc-write (0 B bypassed) + 20 KiB acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 1.31x+1 B (smoothed 1.53x+1 B) + ingested-model 1.50x+1 B (smoothed 1.12x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 195 KiB [≈98 KiB], flushed 2.4 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:171000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.5288076923076923 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:2.55e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:170000 intL0IngestedBytes:30000 intLSMIngestedBytes:30000 intL0WriteAccountedBytes:130000 intIngestedAccountedBytes:20000 intL0WriteLinearModel:{multiplier:1.3076153846153846 constant:1} intL0IngestedLinearModel:{multiplier:1.4995 constant:1} intIngestedLinearModel:{multiplier:1.4995 constant:1} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:171000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.5288076923076923 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:2.55e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:170000 intL0IngestedBytes:30000 intLSMIngestedBytes:30000 intL0WriteAccountedBytes:130000 intIngestedAccountedBytes:20000 intL0WriteLinearModel:{multiplier:1.3076153846153846 constant:1} intL0IngestedLinearModel:{multiplier:1.4995 constant:1} intIngestedLinearModel:{multiplier:1.4995 constant:1} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.53x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -322,7 +322,7 @@ set-state l0-bytes=1000 l0-added-write=191000 l0-added-ingested=30000 l0-files=2 ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 20 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 20 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 1.00x+1 B (smoothed 1.26x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 4.1 KiB, compacted 20 KiB [≈59 KiB], flushed 293 KiB [≈0 B] (mult 1.00); admitting 27 KiB (rate 1.8 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:191000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:27500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:4195} l0WriteLM:{multiplier:1.2641538461538462 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:20000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.9995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:20000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:191000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:27500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:4195} l0WriteLM:{multiplier:1.2641538461538462 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:20000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.9995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:20000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 4195 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.26x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=459(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=459(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -337,7 +337,7 @@ set-state l0-bytes=1000 l0-added-write=211000 l0-added-ingested=30000 l0-files=2 ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 98 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 20 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.50x+1 B (smoothed 0.88x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 3.0 KiB, compacted 20 KiB [≈39 KiB], flushed 293 KiB [≈0 B] (mult 1.00); admitting 23 KiB (rate 1.5 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:211000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:23750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:3097} l0WriteLM:{multiplier:0.8820769230769231 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:100000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:20000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:211000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:23750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:3097} l0WriteLM:{multiplier:0.8820769230769231 constant:1} l0IngestLM:{multiplier:1.125 constant:1} ingestLM:{multiplier:1.2497500000000001 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:300000 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:100000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:20000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 3097 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 0.88x+1 l0-ingest-lm: 1.12x+1 ingest-lm: 1.25x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=396(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=396(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -354,7 +354,7 @@ set-state l0-bytes=1000 l0-added-write=1000 l0-added-ingested=0 l0-files=21 l0-s ---- compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # L0 will see an addition of 200,000 bytes. All were mentioned in the admitted requests. @@ -366,7 +366,7 @@ set-state l0-bytes=1000 l0-added-write=201000 l0-files=21 l0-sublevels=21 print- ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 195 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 1.00x+1 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 195 KiB [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:200000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.99995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:200000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.99995 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -380,7 +380,7 @@ set-state l0-bytes=201000 l0-added-write=401000 l0-files=41 l0-sublevels=41 prin ---- compaction score 2.050[L0-overload] (41 ssts, 41 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting (WAL failover) 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:401000 curL0Bytes:201000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:401000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:1000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:401000 curL0Bytes:201000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:401000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:1000000000 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -398,7 +398,7 @@ set-state l0-bytes=401000 l0-added-write=601000 l0-files=61 l0-sublevels=61 prin ---- compaction score 3.050[L0-overload] (61 ssts, 61 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 2.9 MiB [≈0 B] (mult 1.00); admitting (WAL failover) all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:601000 curL0Bytes:401000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:601000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:2000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:601000 curL0Bytes:401000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:601000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:2000000000 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:3e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -421,7 +421,7 @@ set-state l0-bytes=601000 l0-added-write=801000 l0-files=61 l0-sublevels=5 print ---- compaction score 0.250 (61 ssts, 5 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 0 B [≈98 KiB], flushed 366 KiB [≈0 B] (mult 1.00); admitting (WAL failover) 24 KiB (rate 1.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 4 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:801000 curL0Bytes:601000 cumWriteStallCount:4 cumFlushWriteThroughput:{Bytes:801000 WorkDuration:12000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:3000000000 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:375000 intFlushUtilization:0.4444444444444444 intWriteStalls:4 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:801000 curL0Bytes:601000 cumWriteStallCount:4 cumFlushWriteThroughput:{Bytes:801000 WorkDuration:12000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:3000000000 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:0 intFlushTokens:375000 intFlushUtilization:0.4444444444444444 intWriteStalls:4 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=417(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=417(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -438,11 +438,44 @@ set-state l0-bytes=101000 l0-added-write=1001000 l0-files=10 l0-sublevels=10 pri ---- compaction score 0.500 (10 ssts, 10 sub-levels), L0 growth 195 KiB (write 195 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 195 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 684 KiB [≈391 KiB], flushed 366 KiB [≈0 B] (mult 1.00); admitting (WAL failover) 208 KiB (rate 14 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls -4 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1001000 curL0Bytes:101000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1001000 WorkDuration:20000000000 IdleDuration:420000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:4000000000 smoothedIntL0CompactedBytes:400000 smoothedCompactionByteTokens:212500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:212500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:700000 intFlushTokens:375000 intFlushUtilization:0.4444444444444444 intWriteStalls:-4 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1001000 curL0Bytes:101000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1001000 WorkDuration:20000000000 IdleDuration:420000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:4000000000 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:400000 smoothedCompactionByteTokens:212500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:212500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:700000 intFlushTokens:375000 intFlushUtilization:0.4444444444444444 intWriteStalls:-4 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:200000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:200000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 10000 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=3542(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=3542(elastic 1) max-disk-bw-tokens=unlimited lastTick=false +prep-admission-stats admitted=10000 write-bytes=200000000 +---- +{workCount:10000 writeAccountedBytes:200000000 ingestedAccountedBytes:0 statsToIgnore:{ingestStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0 MemtableOverlappingFiles:0} writeBytes:0} aboveRaftStats:{workCount:10000 writeAccountedBytes:200000000 ingestedAccountedBytes:0} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} + +# unflushed-too-large indicates that the memtable backlog is high. Even though +# the additions to L0 are small compared to write-bytes, the write models are +# not changed. And even though L0 score is 0, elastic tokens are limited. +set-state unflushed-too-large l0-bytes=0 l0-added-write=1002000 l0-files=0 l0-sublevels=0 print-only-first-tick=true wal-secondary-write-sec=1 flush-bytes=1000 flush-work-sec=8 flush-idle-sec=10 write-stall-count=0 +---- +compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 1000 B (flush-backlog) (write 1000 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 9990 (0 bypassed) with 190 MiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 1000 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 100 KiB [≈391 KiB], flushed 1.8 KiB [≈0 B] (mult 1.00); admitting (WAL failover) elastic 100 KiB (rate 6.6 KiB/s) due to L0 growth; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:1002000 curL0Bytes:0 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1002000 WorkDuration:28000000000 IdleDuration:430000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:5000000000 unflushedMemTableTooLarge:true smoothedIntL0CompactedBytes:400000 smoothedCompactionByteTokens:212500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:102000 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:102000 intFlushTokens:1875 intFlushUtilization:0.4444444444444444 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:true perWorkTokensAux:{intWorkCount:9990 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:199800000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +store-request-estimates: writeTokens: 10000 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 +setAvailableTokens: io-tokens=unlimited(elastic 1700) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic 1700) max-disk-bw-tokens=unlimited lastTick=false + +# Small number of additional writes. +prep-admission-stats admitted=10010 write-bytes=200000100 +---- +{workCount:10010 writeAccountedBytes:200000100 ingestedAccountedBytes:0 statsToIgnore:{ingestStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0 MemtableOverlappingFiles:0} writeBytes:0} aboveRaftStats:{workCount:10010 writeAccountedBytes:200000100 ingestedAccountedBytes:0} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}} + +# But the bytes added to L0 are huge. Even though unflushed-too-large is no +# longer true, it was true in the previous call, so the write models are not +# changed, and elastic tokens are limited to the bytes compacted out of L0. +set-state l0-bytes=0 l0-added-write=301002000 l0-files=0 l0-sublevels=0 print-only-first-tick=true wal-secondary-write-sec=1 flush-bytes=1000 flush-work-sec=8 flush-idle-sec=10 write-stall-count=0 +---- +compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 286 MiB (flush-backlog) (write 286 MiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 100 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 286 MiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.37x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 9.8 KiB, compacted 286 MiB [≈143 MiB], flushed 536 MiB [≈0 B] (mult 1.00); admitting (WAL failover) elastic 286 MiB (rate 19 MiB/s) due to L0 growth; write stalls 0 +diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) +{ioLoadListenerState:{cumL0AddedBytes:301002000 curL0Bytes:0 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:301002000 WorkDuration:36000000000 IdleDuration:440000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:6000000000 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:150200000 smoothedCompactionByteTokens:1.5010625e+08 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:300000000 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:10000} l0WriteLM:{multiplier:1.374975 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:300000000 intL0CompactedBytes:300000000 intFlushTokens:5.625e+08 intFlushUtilization:0.4444444444444444 intWriteStalls:0 intWALFailover:true prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:true perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:300000000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:100 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:300000000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +store-request-estimates: writeTokens: 10000 +tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.37x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 +setAvailableTokens: io-tokens=unlimited(elastic 5000000) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic 5000000) max-disk-bw-tokens=unlimited lastTick=false + # Test case with flush tokens. init ---- @@ -455,7 +488,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=1 l0-sublevels=1 print-onl ---- compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # Flush loop utilization is too low for the interval flush tokens to @@ -464,7 +497,7 @@ set-state l0-bytes=10000 l0-added-write=2000 l0-files=1 l0-sublevels=1 flush-byt ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 1000 B (write 1000 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 1000 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 1000 B [≈500 B], flushed 7.3 KiB [≈0 B] (mult 1.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:2000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:2000 WorkDuration:3000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:500 smoothedCompactionByteTokens:500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:2000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:2000 WorkDuration:3000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:500 smoothedCompactionByteTokens:500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -474,7 +507,7 @@ set-state l0-bytes=10000 l0-added-write=3000 l0-files=1 l0-sublevels=1 flush-byt ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 1000 B (write 1000 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 1000 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 1000 B [≈750 B], flushed 7.3 KiB [≈7.3 KiB] (mult 1.00); admitting 7.3 KiB (rate 500 B/s) (elastic 5.9 KiB rate 400 B/s) due to memtable flush (multiplier 1.000) (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:3000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:3000 WorkDuration:5000000000 IdleDuration:210000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:750 smoothedCompactionByteTokens:750 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1 totalNumByteTokens:7500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:6000 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:3000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:3000 WorkDuration:5000000000 IdleDuration:210000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:750 smoothedCompactionByteTokens:750 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1 totalNumByteTokens:7500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:6000 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:1000 intL0CompactedBytes:1000 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:1000 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=125(elastic 100) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=125(elastic 100) max-disk-bw-tokens=unlimited lastTick=false @@ -486,7 +519,7 @@ set-state l0-bytes=10000 l0-added-write=13000 l0-files=1 l0-sublevels=1 flush-by ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈5.2 KiB], flushed 73 KiB [≈40 KiB] (mult 0.97); admitting 39 KiB (rate 2.6 KiB/s) (elastic 31 KiB rate 2.1 KiB/s) due to memtable flush (multiplier 0.975) (used total: 0 B elastic 0 B); write stalls 1 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:13000 curL0Bytes:10000 cumWriteStallCount:1 cumFlushWriteThroughput:{Bytes:13000 WorkDuration:7000000000 IdleDuration:220000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:5375 smoothedCompactionByteTokens:5375 smoothedNumFlushTokens:41250 flushUtilTargetFraction:0.975 totalNumByteTokens:40218 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:32174 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:13000 curL0Bytes:10000 cumWriteStallCount:1 cumFlushWriteThroughput:{Bytes:13000 WorkDuration:7000000000 IdleDuration:220000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:5375 smoothedCompactionByteTokens:5375 smoothedNumFlushTokens:41250 flushUtilTargetFraction:0.975 totalNumByteTokens:40218 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:32174 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=671(elastic 537) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=671(elastic 537) max-disk-bw-tokens=unlimited lastTick=false @@ -498,7 +531,7 @@ set-state l0-bytes=10000 l0-added-write=23000 l0-files=1 l0-sublevels=1 flush-by ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈7.5 KiB], flushed 73 KiB [≈57 KiB] (mult 0.92); admitting 52 KiB (rate 3.5 KiB/s) (elastic 42 KiB rate 2.8 KiB/s) due to memtable flush (multiplier 0.925) (used total: 0 B elastic 0 B); write stalls 2 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:23000 curL0Bytes:10000 cumWriteStallCount:3 cumFlushWriteThroughput:{Bytes:23000 WorkDuration:9000000000 IdleDuration:230000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:7687 smoothedCompactionByteTokens:7687.5 smoothedNumFlushTokens:58125 flushUtilTargetFraction:0.9249999999999999 totalNumByteTokens:53765 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:43012 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:23000 curL0Bytes:10000 cumWriteStallCount:3 cumFlushWriteThroughput:{Bytes:23000 WorkDuration:9000000000 IdleDuration:230000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:7687 smoothedCompactionByteTokens:7687.5 smoothedNumFlushTokens:58125 flushUtilTargetFraction:0.9249999999999999 totalNumByteTokens:53765 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:43012 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=897(elastic 717) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=897(elastic 717) max-disk-bw-tokens=unlimited lastTick=false @@ -509,7 +542,7 @@ set-state l0-bytes=10000 l0-added-write=33000 l0-files=1 l0-sublevels=1 flush-by ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈8.6 KiB], flushed 73 KiB [≈65 KiB] (mult 0.85); admitting 55 KiB (rate 3.7 KiB/s) (elastic 44 KiB rate 2.9 KiB/s) due to memtable flush (multiplier 0.850) (used total: 0 B elastic 0 B); write stalls 5 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:33000 curL0Bytes:10000 cumWriteStallCount:8 cumFlushWriteThroughput:{Bytes:33000 WorkDuration:11000000000 IdleDuration:240000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:8843 smoothedCompactionByteTokens:8843.75 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:0.8499999999999999 totalNumByteTokens:56578 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:45262 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:33000 curL0Bytes:10000 cumWriteStallCount:8 cumFlushWriteThroughput:{Bytes:33000 WorkDuration:11000000000 IdleDuration:240000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:8843 smoothedCompactionByteTokens:8843.75 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:0.8499999999999999 totalNumByteTokens:56578 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:45262 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=943(elastic 755) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=943(elastic 755) max-disk-bw-tokens=unlimited lastTick=false @@ -519,7 +552,7 @@ set-state l0-bytes=10000 l0-added-write=43000 l0-files=1 l0-sublevels=2 flush-by ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.2 KiB], flushed 73 KiB [≈69 KiB] (mult 0.82); admitting 57 KiB (rate 3.8 KiB/s) (elastic 12 KiB rate 785 B/s) due to memtable flush (multiplier 0.825) (used total: 0 B elastic 0 B); write stalls 1 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:43000 curL0Bytes:10000 cumWriteStallCount:9 cumFlushWriteThroughput:{Bytes:43000 WorkDuration:13000000000 IdleDuration:250000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9421 smoothedCompactionByteTokens:9421.875 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:0.8249999999999998 totalNumByteTokens:58394 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11776 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:43000 curL0Bytes:10000 cumWriteStallCount:9 cumFlushWriteThroughput:{Bytes:43000 WorkDuration:13000000000 IdleDuration:250000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9421 smoothedCompactionByteTokens:9421.875 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:0.8249999999999998 totalNumByteTokens:58394 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11776 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=974(elastic 197) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=974(elastic 197) max-disk-bw-tokens=unlimited lastTick=false @@ -534,7 +567,7 @@ set-state l0-bytes=10000 l0-added-write=53000 l0-files=1 l0-sublevels=2 flush-by ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.5 KiB], flushed 73 KiB [≈71 KiB] (mult 1.30); admitting 92 KiB (rate 6.2 KiB/s) (elastic 12 KiB rate 809 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:53000 curL0Bytes:10000 cumWriteStallCount:10 cumFlushWriteThroughput:{Bytes:53000 WorkDuration:15000000000 IdleDuration:260000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9710 smoothedCompactionByteTokens:9710.9375 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3 totalNumByteTokens:94757 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12137 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:53000 curL0Bytes:10000 cumWriteStallCount:10 cumFlushWriteThroughput:{Bytes:53000 WorkDuration:15000000000 IdleDuration:260000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9710 smoothedCompactionByteTokens:9710.9375 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3 totalNumByteTokens:94757 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12137 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1580(elastic 203) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1580(elastic 203) max-disk-bw-tokens=unlimited lastTick=false @@ -545,7 +578,7 @@ set-state l0-bytes=10000 l0-added-write=63000 l0-files=1 l0-sublevels=2 flush-by ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.6 KiB], flushed 73 KiB [≈72 KiB] (mult 1.30); admitting 94 KiB (rate 6.3 KiB/s) (elastic 12 KiB rate 821 B/s) due to memtable flush (multiplier 1.300) (used total: 0 B elastic 0 B); write stalls 1 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:63000 curL0Bytes:10000 cumWriteStallCount:11 cumFlushWriteThroughput:{Bytes:63000 WorkDuration:17000000000 IdleDuration:270000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9855 smoothedCompactionByteTokens:9855.46875 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3 totalNumByteTokens:96128 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12318 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:63000 curL0Bytes:10000 cumWriteStallCount:11 cumFlushWriteThroughput:{Bytes:63000 WorkDuration:17000000000 IdleDuration:270000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9855 smoothedCompactionByteTokens:9855.46875 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3 totalNumByteTokens:96128 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12318 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1603(elastic 206) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1603(elastic 206) max-disk-bw-tokens=unlimited lastTick=false @@ -561,7 +594,7 @@ set-state l0-bytes=10000 l0-added-write=73000 l0-files=1 l0-sublevels=2 flush-by ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 98 KiB (rate 6.5 KiB/s) (elastic 12 KiB rate 827 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 1 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:73000 curL0Bytes:10000 cumWriteStallCount:12 cumFlushWriteThroughput:{Bytes:73000 WorkDuration:19000000000 IdleDuration:280000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9927 smoothedCompactionByteTokens:9927.734375 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12408 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:73000 curL0Bytes:10000 cumWriteStallCount:12 cumFlushWriteThroughput:{Bytes:73000 WorkDuration:19000000000 IdleDuration:280000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9927 smoothedCompactionByteTokens:9927.734375 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12408 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1676(elastic 207) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1676(elastic 207) max-disk-bw-tokens=unlimited lastTick=false @@ -571,7 +604,7 @@ set-state l0-bytes=10000 l0-added-write=83000 l0-files=1 l0-sublevels=2 flush-by ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting elastic 12 KiB (rate 830 B/s) due to L0 growth; write stalls 1 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:83000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:83000 WorkDuration:21000000000 IdleDuration:380000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9963 smoothedCompactionByteTokens:9963.8671875 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12453 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:83000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:83000 WorkDuration:21000000000 IdleDuration:380000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9963 smoothedCompactionByteTokens:9963.8671875 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12453 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic 208) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic 208) max-disk-bw-tokens=unlimited lastTick=false @@ -581,7 +614,7 @@ set-state l0-bytes=10000 l0-added-write=93000 l0-files=1 l0-sublevels=2 flush-by ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.7 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 98 KiB (rate 6.6 KiB/s) (elastic 12 KiB rate 831 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:93000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:93000 WorkDuration:23000000000 IdleDuration:390000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9981 smoothedCompactionByteTokens:9981.93359375 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12476 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:93000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:93000 WorkDuration:23000000000 IdleDuration:390000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9981 smoothedCompactionByteTokens:9981.93359375 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12476 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1682(elastic 208) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1682(elastic 208) max-disk-bw-tokens=unlimited lastTick=false @@ -592,7 +625,7 @@ set-state l0-bytes=10000 l0-added-write=103000 l0-files=1 l0-sublevels=2 flush-b ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.35); admitting 99 KiB (rate 6.6 KiB/s) (elastic 12 KiB rate 832 B/s) due to memtable flush (multiplier 1.350) (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:103000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:103000 WorkDuration:25000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9990 smoothedCompactionByteTokens:9990.966796875 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12487 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:103000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:103000 WorkDuration:25000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9990 smoothedCompactionByteTokens:9990.966796875 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12487 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1685(elastic 209) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1685(elastic 209) max-disk-bw-tokens=unlimited lastTick=false @@ -603,7 +636,7 @@ set-state l0-bytes=10000 l0-added-write=113000 l0-files=1 l0-sublevels=2 flush-b ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.38); admitting 101 KiB (rate 6.7 KiB/s) (elastic 12 KiB rate 832 B/s) due to memtable flush (multiplier 1.375) (used total: 197 KiB elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:113000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:113000 WorkDuration:27000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9995 smoothedCompactionByteTokens:9995.4833984375 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12493 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:202144 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:113000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:113000 WorkDuration:27000000000 IdleDuration:410000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9995 smoothedCompactionByteTokens:9995.4833984375 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12493 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:202144 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1718(elastic 209) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1718(elastic 209) max-disk-bw-tokens=unlimited lastTick=false @@ -614,7 +647,7 @@ set-state l0-bytes=10000 l0-added-write=123000 l0-files=1 l0-sublevels=2 flush-b ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.40); admitting 102 KiB (rate 6.8 KiB/s) (elastic 12 KiB rate 833 B/s) due to memtable flush (multiplier 1.400) (used total: 201 KiB elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:123000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:123000 WorkDuration:29000000000 IdleDuration:420000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9997 smoothedCompactionByteTokens:9997.74169921875 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12496 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:206068 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:123000 curL0Bytes:10000 cumWriteStallCount:13 cumFlushWriteThroughput:{Bytes:123000 WorkDuration:29000000000 IdleDuration:420000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9997 smoothedCompactionByteTokens:9997.74169921875 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12496 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 intWALFailover:false prevTokensUsed:206068 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1750(elastic 209) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1750(elastic 209) max-disk-bw-tokens=unlimited lastTick=false @@ -625,7 +658,7 @@ set-state l0-bytes=10000 l0-added-write=133000 l0-files=1 l0-sublevels=2 flush-b ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 9.8 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈9.8 KiB], flushed 73 KiB [≈73 KiB] (mult 1.38); admitting 101 KiB (rate 6.7 KiB/s) (elastic 12 KiB rate 833 B/s) due to memtable flush (multiplier 1.375) (used total: 205 KiB elastic 0 B); write stalls 1 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:133000 curL0Bytes:10000 cumWriteStallCount:14 cumFlushWriteThroughput:{Bytes:133000 WorkDuration:31000000000 IdleDuration:430000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:9998 smoothedCompactionByteTokens:9998.870849609375 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12497 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:209906 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:133000 curL0Bytes:10000 cumWriteStallCount:14 cumFlushWriteThroughput:{Bytes:133000 WorkDuration:31000000000 IdleDuration:430000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:9998 smoothedCompactionByteTokens:9998.870849609375 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:12497 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 intWALFailover:false prevTokensUsed:209906 prevTokensUsedByElasticWork:0 tokenKind:1 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1719(elastic 209) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1719(elastic 209) max-disk-bw-tokens=unlimited lastTick=false @@ -643,7 +676,7 @@ set-state l0-bytes=0 l0-added-write=0 bytes-read=0 bytes-written=0 provisioned-b ---- compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:0 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:0 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # Admit some work. @@ -657,7 +690,7 @@ set-state l0-bytes=10 l0-added-write=10 bytes-read=60 bytes-written=50 provision ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 10 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 150 B [≈0 B] (mult 1.35); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization +Inf, tokensUsed (elastic 0 B, snapshot 100 B, regular 100 B) tokens (write 1.4 KiB (prev 0 B), read 60 B (prev 0 B)), writeBW 3 B/s, readBW 4 B/s, provisioned 100 B/s) -{ioLoadListenerState:{cumL0AddedBytes:10 curL0Bytes:10 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:10 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:60 bytesWritten:50} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:1440 diskWriteTokensAllocated:0 diskReadTokens:60 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10 intL0CompactedBytes:0 intFlushTokens:150 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:10 curL0Bytes:10 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:10 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:60 bytesWritten:50} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:1440 diskWriteTokensAllocated:0 diskReadTokens:60 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:10 intL0CompactedBytes:0 intFlushTokens:150 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=24 read-bw-tokens=1 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=24 lastTick=false @@ -670,7 +703,7 @@ set-state l0-bytes=40 l0-added-write=40 bytes-read=120 bytes-written=400 provisi ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 30 B (write 30 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10 (0 bypassed) with 10 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 30 B adjusted-LSM-writes + 350 B adjusted-disk-writes + write-model 2.00x+1 B (smoothed 1.88x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 11.33x+1 B (smoothed 30.92x+1 B) + at-admission-tokens 2 B, compacted 0 B [≈0 B], flushed 435 B [≈0 B] (mult 1.35); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.28, tokensUsed (elastic 0 B, snapshot 200 B, regular 200 B) tokens (write 1.4 KiB (prev 1.4 KiB), read 60 B (prev 60 B)), writeBW 23 B/s, readBW 4 B/s, provisioned 100 B/s) -{ioLoadListenerState:{cumL0AddedBytes:40 curL0Bytes:40 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:40 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:120 bytesWritten:400} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:1440 diskWriteTokensAllocated:0 diskReadTokens:60 diskReadTokensAllocated:0} requestEstimates:{writeTokens:2} l0WriteLM:{multiplier:1.875 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:30.916666666666668 constant:1} aux:{intL0AddedBytes:30 intL0CompactedBytes:0 intFlushTokens:435 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:30 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:10 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:11.333333333333334 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:30 intAdjustedDiskWriteBytes:350} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:40 curL0Bytes:40 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:40 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:120 bytesWritten:400} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:1440 diskWriteTokensAllocated:0 diskReadTokens:60 diskReadTokensAllocated:0} requestEstimates:{writeTokens:2} l0WriteLM:{multiplier:1.875 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:30.916666666666668 constant:1} aux:{intL0AddedBytes:30 intL0CompactedBytes:0 intFlushTokens:435 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:30 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:10 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:11.333333333333334 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:30 intAdjustedDiskWriteBytes:350} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 2 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.88x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 30.92x+1 setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=24 read-bw-tokens=1 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=24 lastTick=false @@ -683,7 +716,7 @@ set-state l0-bytes=90 l0-added-write=80 bytes-read=180 bytes-written=1200 provis ---- compaction score 0.100 (1 ssts, 2 sub-levels), L0 growth 40 B (write 40 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 40 (0 bypassed) with 50 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 40 B adjusted-LSM-writes + 800 B adjusted-disk-writes + write-model 0.50x+1 B (smoothed 1.19x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 19.00x+1 B (smoothed 24.96x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 600 B [≈0 B] (mult 1.35); admitting elastic 1 B (rate 0 B/s) due to L0 growth; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.62, tokensUsed (elastic 0 B, snapshot 500 B, regular 400 B) tokens (write 1.4 KiB (prev 1.4 KiB), read 60 B (prev 60 B)), writeBW 53 B/s, readBW 4 B/s, provisioned 100 B/s) -{ioLoadListenerState:{cumL0AddedBytes:80 curL0Bytes:90 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:80 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:180 bytesWritten:1200} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:1440 diskWriteTokensAllocated:0 diskReadTokens:60 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.1875 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:24.958333333333336 constant:1} aux:{intL0AddedBytes:40 intL0CompactedBytes:0 intFlushTokens:600 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:40 intL0WriteBytes:40 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:50 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:19 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:40 intAdjustedDiskWriteBytes:800} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:80 curL0Bytes:90 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:80 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:180 bytesWritten:1200} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:1440 diskWriteTokensAllocated:0 diskReadTokens:60 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.1875 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:24.958333333333336 constant:1} aux:{intL0AddedBytes:40 intL0CompactedBytes:0 intFlushTokens:600 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:40 intL0WriteBytes:40 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:50 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0.5 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:19 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:40 intAdjustedDiskWriteBytes:800} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.19x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 24.96x+1 setAvailableTokens: io-tokens=unlimited(elastic 1) elastic-disk-bw-tokens=24 read-bw-tokens=1 max-byte-tokens=unlimited(elastic 1) max-disk-bw-tokens=24 lastTick=false @@ -701,7 +734,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=21 l0-sublevels=21 print-o ---- compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false prep-admission-stats admitted=10000 write-bytes=40000 @@ -716,7 +749,7 @@ set-state l0-bytes=10000 l0-added-write=101000 l0-files=21 l0-sublevels=21 print ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:5} l0WriteLM:{multiplier:2 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:2.25 constant:1} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 5 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 2.00x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -729,14 +762,14 @@ set-state l0-bytes=100 l0-added-write=0 bytes-read=0 bytes-written=0 provisioned ---- compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:0 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false set-state l0-bytes=100 l0-added-write=100000 bytes-read=1000000 bytes-written=2000000 provisioned-bandwidth=10 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true loaded=true ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization NaN, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 977 KiB (prev 0 B)), writeBW 130 KiB/s, readBW 65 KiB/s, provisioned 10 B/s) -{ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:100000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:1000000 bytesWritten:2000000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:0 diskWriteTokensAllocated:0 diskReadTokens:1000000 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:100000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:1000000 bytesWritten:2000000} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:0 diskWriteTokensAllocated:0 diskReadTokens:1000000 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=0 read-bw-tokens=67 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=0 lastTick=false @@ -753,7 +786,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=5 l0-sublevels=5 print-onl ---- compaction score 0.000 (5 ssts, 5 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # No restricting of tokens. @@ -765,7 +798,7 @@ set-state l0-bytes=10000 l0-added-write=501000 l0-files=1 l0-sublevels=1 print-o ---- compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 488 KiB (write 488 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 488 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 488 KiB [≈244 KiB], flushed 7.2 MiB [≈0 B] (mult 1.35); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:250000 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:500000 intL0CompactedBytes:500000 intFlushTokens:7.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:500000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:500000 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:250000 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:500000 intL0CompactedBytes:500000 intFlushTokens:7.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:500000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:500000 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false @@ -782,7 +815,7 @@ set-state l0-bytes=10000 l0-added-write=501002 l0-files=10 l0-sublevels=10 print ---- compaction score 0.500 (10 ssts, 10 sub-levels), L0 growth 2 B (write 2 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 2 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 2 B [≈122 KiB], flushed 30 B [≈0 B] (mult 1.35); admitting 183 KiB (rate 12 KiB/s) (elastic 30 KiB rate 2.0 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:501002 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501002 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:125001 smoothedCompactionByteTokens:187500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:187500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:31250 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:2 intL0CompactedBytes:2 intFlushTokens:30 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:2 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:501002 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501002 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:125001 smoothedCompactionByteTokens:187500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:187500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:31250 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:2 intL0CompactedBytes:2 intFlushTokens:30 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:2 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=13(elastic 3) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=3125(elastic 521) max-disk-bw-tokens=unlimited lastTick=false @@ -796,7 +829,7 @@ set-state l0-bytes=10000 l0-added-write=501000 l0-files=10 l0-sublevels=15 print ---- compaction score 0.750 (10 ssts, 15 sub-levels), L0 growth 0 B (write -2 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈61 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 114 KiB (rate 7.6 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:62500 smoothedCompactionByteTokens:117187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:117187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:-2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:62500 smoothedCompactionByteTokens:117187.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:117187 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:-2 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=8(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1954(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -811,7 +844,7 @@ set-state l0-bytes=10000 l0-added-write=501000 l0-files=10 l0-sublevels=20 print ---- compaction score 1.000 (10 ssts, 20 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈30 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 65 KiB (rate 4.3 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:31250 smoothedCompactionByteTokens:66406.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66406 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:31250 smoothedCompactionByteTokens:66406.25 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66406 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=5(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1107(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -824,7 +857,7 @@ set-state l0-bytes=10000 l0-added-write=501000 l0-files=10 l0-sublevels=5 print- ---- compaction score 0.250 (10 ssts, 5 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈15 KiB], flushed 0 B [≈0 B] (mult 1.35); admitting 48 KiB (rate 3.2 KiB/s) (elastic 13 KiB rate 911 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:15625 smoothedCompactionByteTokens:48828.125 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:48828 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:13671 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:501000 WorkDuration:6000000000 IdleDuration:600000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:15625 smoothedCompactionByteTokens:48828.125 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:48828 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:13671 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=4(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=814(elastic 228) max-disk-bw-tokens=unlimited lastTick=false @@ -841,7 +874,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=21 l0-sublevels=21 print-o ---- compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # Score is 21/20. @@ -849,7 +882,7 @@ set-state l0-bytes=10000 l0-added-write=101000 l0-files=21 l0-sublevels=21 print ---- compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 12 KiB (rate 833 B/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:101000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=209(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=209(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -862,7 +895,7 @@ set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21 print ---- compaction score 0.350 (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 98 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈73 KiB], flushed 1.4 MiB [≈0 B] (mult 1.35); admitting 65 KiB (rate 4.3 KiB/s) (elastic 46 KiB rate 3.1 KiB/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:66250 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66250 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:46875 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:201000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:66250 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:66250 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:46875 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:1.5e+06 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:100000 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=1105(elastic 782) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=1105(elastic 782) max-disk-bw-tokens=unlimited lastTick=false @@ -872,7 +905,7 @@ set-state l0-bytes=50000 l0-added-write=260994 l0-files=21 l0-sublevels=21 print ---- compaction score 0.500 (21 ssts, 21 sub-levels), L0 growth 59 KiB (write 59 KiB (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 59 KiB adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 20 KiB [≈46 KiB], flushed 879 KiB [≈0 B] (mult 1.35); admitting 56 KiB (rate 3.7 KiB/s) (elastic 12 KiB rate 791 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:260994 curL0Bytes:50000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:260994 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:47497 smoothedCompactionByteTokens:56873 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:56873 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11874 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:59994 intL0CompactedBytes:19994 intFlushTokens:899910 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:59994 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:59994 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:260994 curL0Bytes:50000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:260994 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:47497 smoothedCompactionByteTokens:56873 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:56873 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:11874 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:59994 intL0CompactedBytes:19994 intFlushTokens:899910 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:59994 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:59994 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=948(elastic 198) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=948(elastic 198) max-disk-bw-tokens=unlimited lastTick=false @@ -889,7 +922,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=6 print-o ---- compaction score 0.000 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 0.00); admitting all; write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:1000000000 IdleDuration:100000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:0 numOutLevelsGauge:1} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:9223372036854775807 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:false} ioThreshold:} tick: 0, setAvailableTokens: io-tokens=unlimited(elastic unlimited) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=unlimited(elastic unlimited) max-disk-bw-tokens=unlimited lastTick=false # No compactions out of L0, but L0 token lower bound is set to 1MB/2 and then smoothed to give us "admitting 244 KiB (rate 16 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound)" @@ -897,7 +930,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=6 base-le ---- compaction score 0.300 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 244 KiB (rate 16 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:1000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:250000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:2000000000 IdleDuration:200000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:1000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:250000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:250000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=4167(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=4167(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -907,7 +940,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=6 base-le ---- compaction score 0.300 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 366 KiB (rate 24 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:2000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:375000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:375000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:3000000000 IdleDuration:300000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:2000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:375000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:375000 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=6250(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=6250(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -917,7 +950,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=6 base-le ---- compaction score 0.300 (100 ssts, 6 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 427 KiB (rate 28 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth(used token lower bound) (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:437500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:437500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:4000000000 IdleDuration:400000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:437500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:437500 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:true recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=7292(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=7292(elastic 1) max-disk-bw-tokens=unlimited lastTick=false @@ -930,7 +963,7 @@ set-state l0-bytes=10000 l0-added-write=1000 l0-files=100 l0-sublevels=10 base-l ---- compaction score 0.500 (100 ssts, 10 sub-levels), L0 growth 0 B (write 0 B (ignored 0 B) ingest 0 B (ignored 0 B)): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + 0 B adjusted-LSM-writes + 0 B adjusted-disk-writes + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + write-amp-model 0.00x+0 B (smoothed 50.50x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B] (mult 1.35); admitting 214 KiB (rate 14 KiB/s) (elastic 1 B rate 0 B/s) due to L0 growth (used total: 0 B elastic 0 B); write stalls 0 diskBandwidthLimiter (tokenUtilization 0.00, tokensUsed (elastic 0 B, snapshot 0 B, regular 0 B) tokens (write 0 B (prev 0 B), read 0 B (prev 0 B)), writeBW 0 B/s, readBW 0 B/s, provisioned 0 B/s) -{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:218750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:218750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} +{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 cumFlushWriteThroughput:{Bytes:1000 WorkDuration:5000000000 IdleDuration:500000000000} diskBW:{bytesRead:0 bytesWritten:0} cumCompactionStats:{writeBytes:3000000 numOutLevelsGauge:2} cumWALSecondaryWriteDuration:0 unflushedMemTableTooLarge:false smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:218750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.35 totalNumByteTokens:218750 byteTokensAllocated:0 byteTokensUsed:0 byteTokensUsedByElasticWork:0 totalNumElasticByteTokens:1 elasticByteTokensAllocated:0 diskWriteTokens:9223372036854775807 diskWriteTokensAllocated:0 diskReadTokens:0 diskReadTokensAllocated:0} requestEstimates:{writeTokens:1} l0WriteLM:{multiplier:1.75 constant:1} l0IngestLM:{multiplier:0.7505 constant:1} ingestLM:{multiplier:1 constant:1} writeAmpLM:{multiplier:50.5 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0.009900990099009901 intWriteStalls:0 intWALFailover:false prevTokensUsed:0 prevTokensUsedByElasticWork:0 tokenKind:0 usedCompactionTokensLowerBound:false recentUnflushedMemTableTooLarge:false perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:0 intAdjustedDiskWriteBytes:0} doLogFlush:true} ioThreshold:} store-request-estimates: writeTokens: 1 tick: 0, setAdmittedDoneModelsLocked: l0-write-lm: 1.75x+1 l0-ingest-lm: 0.75x+1 ingest-lm: 1.00x+1 write-amp-lm: 50.50x+1 setAvailableTokens: io-tokens=3646(elastic 1) elastic-disk-bw-tokens=unlimited read-bw-tokens=0 max-byte-tokens=3646(elastic 1) max-disk-bw-tokens=unlimited lastTick=false diff --git a/pkg/util/admission/testdata/store_per_work_token_estimator b/pkg/util/admission/testdata/store_per_work_token_estimator index 1e55b5859926..97e7c3e5445b 100644 --- a/pkg/util/admission/testdata/store_per_work_token_estimator +++ b/pkg/util/admission/testdata/store_per_work_token_estimator @@ -121,6 +121,27 @@ L0-ingest-tokens: int: 0.00x+0 smoothed: 0.84x+1 per-work-accounted: 25165 ingest-tokens: int: 0.00x+0 smoothed: 1.07x+3 per-work-accounted: 25165 write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 +# Flushed is tiny compared to write-accounted, but since unflushed-too-large +# is true, the at-admission-tokens and L0-write-tokens are not updated. +update flushed=10 ingested=0 admitted=100 write-accounted=100000 ingested-accounted=0 above-raft-count=10 above-raft-write=2000 unflushed-too-large +---- +interval state: {intWorkCount:100 intL0WriteBytes:10 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:100000 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10 intAdjustedDiskWriteBytes:0} +at-admission-tokens: 64 +L0-write-tokens: int: 0.00x+0 smoothed: 2.06x+1 per-work-accounted: 72 +L0-ingest-tokens: int: 0.00x+0 smoothed: 0.84x+1 per-work-accounted: 25165 +ingest-tokens: int: 0.00x+0 smoothed: 1.07x+1 per-work-accounted: 25165 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 + +# Flushed is huge compared to write-accounted, but since unflushed-too-large +# is true, the at-admission-tokens and L0-write-tokens are not updated. +update flushed=10000000 ingested=0 admitted=50 write-accounted=100 ingested-accounted=0 above-raft-count=20 above-raft-write=40 unflushed-too-large +---- +interval state: {intWorkCount:50 intL0WriteBytes:10000000 intL0IngestedBytes:0 intLSMIngestedBytes:0 intL0WriteAccountedBytes:100 intIngestedAccountedBytes:0 intL0WriteLinearModel:{multiplier:0 constant:0} intL0IngestedLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intWriteAmpLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredWriteBytes:0 intL0IgnoredIngestedBytes:0 intAdjustedLSMWrites:10000000 intAdjustedDiskWriteBytes:0} +at-admission-tokens: 64 +L0-write-tokens: int: 0.00x+0 smoothed: 2.06x+1 per-work-accounted: 72 +L0-ingest-tokens: int: 0.00x+0 smoothed: 0.84x+1 per-work-accounted: 25165 +ingest-tokens: int: 0.00x+0 smoothed: 1.07x+1 per-work-accounted: 25165 +write-amp: int: 0.00x+0 smoothed: 50.50x+1 per-work-accounted: 1 # Test write amp LM From 34e39c0ca538f517a07117ca56e44e5ac43240c3 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Sat, 11 Jan 2025 16:35:13 -0500 Subject: [PATCH 043/126] opt: reduce allocations when filtering histogram buckets `cat.HistogramBuckets` are now returned and passed by value in `getFilteredBucket` and `(*Histogram).addBucket`, respectively, eliminating some heap allocations. Also, two allocations when building spans from buckets via the `spanBuilder` have been combined into one. The new `(*spanBuilder).init` method simplifies the API by no longer requiring that prefix datums are passed to every invocation of `makeSpanFromBucket`. This also reduces redundant copying of the prefix. Release note: None --- pkg/sql/opt/props/histogram.go | 44 +++++++++++++++++------------ pkg/sql/opt/props/histogram_test.go | 4 +-- 2 files changed, 28 insertions(+), 20 deletions(-) diff --git a/pkg/sql/opt/props/histogram.go b/pkg/sql/opt/props/histogram.go index ddda074f426f..762b6e29d2f9 100644 --- a/pkg/sql/opt/props/histogram.go +++ b/pkg/sql/opt/props/histogram.go @@ -336,10 +336,11 @@ func (h *Histogram) filter( // used for comparison and are not stored, and two spans are never // built and referenced simultaneously. var sb spanBuilder + sb.init(prefix) { // Limit the scope of firstBucket to avoid referencing it below after // sb.makeSpanFromBucket has been called again. - firstBucket := sb.makeSpanFromBucket(ctx, &iter, prefix) + firstBucket := sb.makeSpanFromBucket(ctx, &iter) for spanIndex < spanCount { span := getSpan(spanIndex) if firstBucket.StartsAfter(&keyCtx, span) { @@ -357,7 +358,7 @@ func (h *Histogram) filter( span := getSpan(spanIndex) bucIndex := sort.Search(bucketCount, func(i int) bool { iter.setIdx(i) - bucket := sb.makeSpanFromBucket(ctx, &iter, prefix) + bucket := sb.makeSpanFromBucket(ctx, &iter) if desc { return span.StartsAfter(&keyCtx, &bucket) } @@ -382,7 +383,7 @@ func (h *Histogram) filter( } if spanCount == 1 && bucIndex < bucketCount-1 { iter.setIdx(bucIndex + 1) - bucket := sb.makeSpanFromBucket(ctx, &iter, prefix) + bucket := sb.makeSpanFromBucket(ctx, &iter) if !desc && bucket.StartsAfter(&keyCtx, span) || desc && !bucket.StartsAfter(&keyCtx, span) { newBucketCount = 2 @@ -406,7 +407,7 @@ func (h *Histogram) filter( // Convert the bucket to a span in order to take advantage of the // constraint library. - left := sb.makeSpanFromBucket(ctx, &iter, prefix) + left := sb.makeSpanFromBucket(ctx, &iter) right := getSpan(spanIndex) if left.StartsAfter(&keyCtx, right) { @@ -425,7 +426,7 @@ func (h *Histogram) filter( continue } - filteredBucket := iter.b + filteredBucket := *iter.b if filteredSpan.Compare(&keyCtx, &left) != 0 { // The bucket was cut off in the middle. Get the resulting filtered // bucket. @@ -476,7 +477,7 @@ func (h *Histogram) filter( filtered.addEmptyBucket(ctx, iter.inclusiveLowerBound(ctx), desc) } else if lastBucket := filtered.buckets[len(filtered.buckets)-1]; lastBucket.NumRange != 0 { iter.setIdx(0) - span := sb.makeSpanFromBucket(ctx, &iter, prefix) + span := sb.makeSpanFromBucket(ctx, &iter) ub := h.getPrevUpperBound(ctx, span.EndKey(), span.EndBoundary(), colOffset) filtered.addEmptyBucket(ctx, ub, desc) } @@ -567,16 +568,16 @@ func (h *Histogram) getPrevUpperBound( } func (h *Histogram) addEmptyBucket(ctx context.Context, upperBound tree.Datum, desc bool) { - h.addBucket(ctx, &cat.HistogramBucket{UpperBound: upperBound}, desc) + h.addBucket(ctx, cat.HistogramBucket{UpperBound: upperBound}, desc) } -func (h *Histogram) addBucket(ctx context.Context, bucket *cat.HistogramBucket, desc bool) { +func (h *Histogram) addBucket(ctx context.Context, bucket cat.HistogramBucket, desc bool) { // Check whether we can combine this bucket with the previous bucket. if len(h.buckets) != 0 { lastBucket := &h.buckets[len(h.buckets)-1] - lower, higher := lastBucket, bucket + lower, higher := lastBucket, &bucket if desc { - lower, higher = bucket, lastBucket + lower, higher = &bucket, lastBucket } if lower.NumRange == 0 && lower.NumEq == 0 && higher.NumRange == 0 { lastBucket.NumEq = higher.NumEq @@ -592,7 +593,7 @@ func (h *Histogram) addBucket(ctx context.Context, bucket *cat.HistogramBucket, return } } - h.buckets = append(h.buckets, *bucket) + h.buckets = append(h.buckets, bucket) } // ApplySelectivity returns a histogram with the given selectivity applied. If @@ -740,6 +741,15 @@ type spanBuilder struct { endScratch []tree.Datum } +func (sb *spanBuilder) init(prefix []tree.Datum) { + n := len(prefix) + 1 + d := make([]tree.Datum, 2*n) + copy(d, prefix) + copy(d[n:], prefix) + sb.startScratch = d[:n:n] + sb.endScratch = d[n:] +} + // makeSpanFromBucket constructs a constraint.Span from iter's current histogram // bucket. // @@ -747,7 +757,7 @@ type spanBuilder struct { // on the same spanBuilder. This is because it reuses scratch slices in the // spanBuilder to reduce allocations when building span keys. func (sb *spanBuilder) makeSpanFromBucket( - ctx context.Context, iter *histogramIter, prefix []tree.Datum, + ctx context.Context, iter *histogramIter, ) (span constraint.Span) { start, startBoundary := iter.lowerBound() end, endBoundary := iter.upperBound() @@ -762,10 +772,8 @@ func (sb *spanBuilder) makeSpanFromBucket( startBoundary = constraint.IncludeBoundary endBoundary = constraint.IncludeBoundary } - sb.startScratch = append(sb.startScratch[:0], prefix...) - sb.startScratch = append(sb.startScratch, start) - sb.endScratch = append(sb.endScratch[:0], prefix...) - sb.endScratch = append(sb.endScratch, end) + sb.startScratch[len(sb.startScratch)-1] = start + sb.endScratch[len(sb.endScratch)-1] = end span.Init( constraint.MakeCompositeKey(sb.startScratch...), startBoundary, @@ -808,7 +816,7 @@ func (sb *spanBuilder) makeSpanFromBucket( // we use the heuristic that NumRange is reduced by half. func getFilteredBucket( iter *histogramIter, keyCtx *constraint.KeyContext, filteredSpan *constraint.Span, colOffset int, -) *cat.HistogramBucket { +) cat.HistogramBucket { spanLowerBound := filteredSpan.StartKey().Value(colOffset) spanUpperBound := filteredSpan.EndKey().Value(colOffset) bucketLowerBound := iter.inclusiveLowerBound(keyCtx.Ctx) @@ -915,7 +923,7 @@ func getFilteredBucket( if iter.desc { upperBound = spanLowerBound } - return &cat.HistogramBucket{ + return cat.HistogramBucket{ NumEq: numEq, NumRange: numRange, DistinctRange: distinctCountRange, diff --git a/pkg/sql/opt/props/histogram_test.go b/pkg/sql/opt/props/histogram_test.go index ecb4da360fa6..f5153ec6d36f 100644 --- a/pkg/sql/opt/props/histogram_test.go +++ b/pkg/sql/opt/props/histogram_test.go @@ -438,8 +438,8 @@ func TestFilterBucket(t *testing.T) { // the second bucket. iter.setIdx(1) b := getFilteredBucket(&iter, &keyCtx, span, colOffset) - roundBucket(b) - return b, nil + roundBucket(&b) + return &b, nil } runTest := func(h *Histogram, testData []testCase, colOffset int, typs ...types.Family) { From 0997705533a8e7045d3bc720e63432c54145a630 Mon Sep 17 00:00:00 2001 From: David Taylor Date: Tue, 31 Dec 2024 21:48:28 +0000 Subject: [PATCH 044/126] server/admin: add job messages to job detail resp This should allow showing them on the job detail page. Release note: none. Epic: none. --- docs/generated/http/full.md | 32 ++++++++++++++++++++++++ pkg/ccl/serverccl/admin_test.go | 3 +++ pkg/server/admin.go | 43 +++++++++++++++++++++++++++++++++ pkg/server/serverpb/admin.proto | 8 ++++++ 4 files changed, 86 insertions(+) diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index d6c26b5ec253..95d39c4cb5f7 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -6645,6 +6645,7 @@ JobResponse contains the job record for a job. | num_runs | [int64](#cockroach.server.serverpb.JobsResponse-int64) | | | [reserved](#support-status) | | execution_failures | [JobResponse.ExecutionFailure](#cockroach.server.serverpb.JobsResponse-cockroach.server.serverpb.JobResponse.ExecutionFailure) | repeated | ExecutionFailures is a log of execution failures of the job. It is not guaranteed to contain all execution failures and some execution failures may not contain an error or end. | [reserved](#support-status) | | coordinator_id | [int64](#cockroach.server.serverpb.JobsResponse-int64) | | coordinator_id identifies the node coordinating the job. This value will only be present for jobs that are currently running or recently ran. | [reserved](#support-status) | +| messages | [JobMessage](#cockroach.server.serverpb.JobsResponse-cockroach.server.serverpb.JobMessage) | repeated | | [reserved](#support-status) | @@ -6667,6 +6668,21 @@ attempt starting at start and ending at end. + +#### JobMessage + + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| kind | [string](#cockroach.server.serverpb.JobsResponse-string) | | | [reserved](#support-status) | +| timestamp | [google.protobuf.Timestamp](#cockroach.server.serverpb.JobsResponse-google.protobuf.Timestamp) | | | [reserved](#support-status) | +| message | [string](#cockroach.server.serverpb.JobsResponse-string) | | | [reserved](#support-status) | + + + + + ## Job @@ -6725,6 +6741,7 @@ JobResponse contains the job record for a job. | num_runs | [int64](#cockroach.server.serverpb.JobResponse-int64) | | | [reserved](#support-status) | | execution_failures | [JobResponse.ExecutionFailure](#cockroach.server.serverpb.JobResponse-cockroach.server.serverpb.JobResponse.ExecutionFailure) | repeated | ExecutionFailures is a log of execution failures of the job. It is not guaranteed to contain all execution failures and some execution failures may not contain an error or end. | [reserved](#support-status) | | coordinator_id | [int64](#cockroach.server.serverpb.JobResponse-int64) | | coordinator_id identifies the node coordinating the job. This value will only be present for jobs that are currently running or recently ran. | [reserved](#support-status) | +| messages | [JobMessage](#cockroach.server.serverpb.JobResponse-cockroach.server.serverpb.JobMessage) | repeated | | [reserved](#support-status) | @@ -6748,6 +6765,21 @@ attempt starting at start and ending at end. + +#### JobMessage + + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| kind | [string](#cockroach.server.serverpb.JobResponse-string) | | | [reserved](#support-status) | +| timestamp | [google.protobuf.Timestamp](#cockroach.server.serverpb.JobResponse-google.protobuf.Timestamp) | | | [reserved](#support-status) | +| message | [string](#cockroach.server.serverpb.JobResponse-string) | | | [reserved](#support-status) | + + + + + ## Locations diff --git a/pkg/ccl/serverccl/admin_test.go b/pkg/ccl/serverccl/admin_test.go index 8c228a054670..b454d6a5b968 100644 --- a/pkg/ccl/serverccl/admin_test.go +++ b/pkg/ccl/serverccl/admin_test.go @@ -151,6 +151,9 @@ func TestAdminAPIJobs(t *testing.T) { err = getAdminJSONProto(s, path, &jobRes) require.NoError(t, err) + // Messages are not equal, since they only appear in the single job response, + // so the deep-equal check would fail; copy it so the overall check passes. + jobRes.Messages = backups[0].Messages require.Equal(t, backups[0], jobRes) } diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 035c386a890d..cc7d3c556889 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -20,6 +20,7 @@ import ( apd "github.com/cockroachdb/apd/v3" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -2473,9 +2474,51 @@ func jobHelper( return nil, err } + // On 25.1+, add any recorded job messages to the response as well. + if sqlServer.cfg.Settings.Version.IsActive(ctx, clusterversion.V25_1) { + job.Messages = fetchJobMessages(ctx, job.ID, userName, sqlServer) + } return &job, nil } +func fetchJobMessages( + ctx context.Context, jobID int64, user username.SQLUsername, sqlServer *SQLServer, +) (messages []serverpb.JobMessage) { + const msgQuery = `SELECT kind, written, message FROM system.job_message WHERE job_id = $1 ORDER BY written DESC` + it, err := sqlServer.internalExecutor.QueryIteratorEx(ctx, "admin-job-messages", nil, + sessiondata.InternalExecutorOverride{User: user}, + msgQuery, + jobID, + ) + + if err != nil { + return []serverpb.JobMessage{{Kind: "error", Timestamp: timeutil.Now(), Message: err.Error()}} + } + + defer func() { + if err := it.Close(); err != nil { + messages = []serverpb.JobMessage{{Kind: "error", Timestamp: timeutil.Now(), Message: err.Error()}} + } + }() + + for { + ok, err := it.Next(ctx) + if err != nil { + return []serverpb.JobMessage{{Kind: "error", Timestamp: timeutil.Now(), Message: err.Error()}} + } + if !ok { + break + } + row := it.Cur() + messages = append(messages, serverpb.JobMessage{ + Kind: string(tree.MustBeDStringOrDNull(row[0])), + Timestamp: tree.MustBeDTimestampTZ(row[1]).Time, + Message: string(tree.MustBeDStringOrDNull(row[2])), + }) + } + return messages +} + func (s *adminServer) Locations( ctx context.Context, req *serverpb.LocationsRequest, ) (_ *serverpb.LocationsResponse, retErr error) { diff --git a/pkg/server/serverpb/admin.proto b/pkg/server/serverpb/admin.proto index 2a320cabb54d..a9e068884e0e 100644 --- a/pkg/server/serverpb/admin.proto +++ b/pkg/server/serverpb/admin.proto @@ -687,6 +687,12 @@ message JobRequest { int64 job_id = 1; } +message JobMessage { + string kind = 1; + google.protobuf.Timestamp timestamp = 2 [(gogoproto.nullable)=false, (gogoproto.stdtime) = true]; + string message = 3; +} + // JobResponse contains the job record for a job. message JobResponse { int64 id = 1 [(gogoproto.customname) = "ID"]; @@ -737,6 +743,8 @@ message JobResponse { // coordinator_id identifies the node coordinating the job. This value will // only be present for jobs that are currently running or recently ran. int64 coordinator_id = 21 [(gogoproto.customname) = "CoordinatorID"]; + + repeated JobMessage messages = 22 [(gogoproto.nullable)=false]; } // LocationsRequest requests system locality location information. From ba6e283392b3b6edeb2489aa0b9a03a8a3f7a700 Mon Sep 17 00:00:00 2001 From: Silvano Ravotto Date: Sat, 11 Jan 2025 14:25:19 -0500 Subject: [PATCH 045/126] roachtest: cleaning up microceph cluster The microceph cluster must be removed at the end of test to allow for the node to be reused. Epic: none Fixes: #138860 Release note: None --- .../tests/s3_clone_backup_restore.go | 1 + pkg/cmd/roachtest/tests/s3_microceph.go | 21 ++++++++++++++++++- 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/s3_clone_backup_restore.go b/pkg/cmd/roachtest/tests/s3_clone_backup_restore.go index 4e64ad1e55c7..a91e0a5087fb 100644 --- a/pkg/cmd/roachtest/tests/s3_clone_backup_restore.go +++ b/pkg/cmd/roachtest/tests/s3_clone_backup_restore.go @@ -60,6 +60,7 @@ func registerBackupS3Clones(r registry.Registry) { version: cephVersion, } ceph.install(ctx) + defer ceph.cleanup(ctx) v.validateBackupRestore(ctx, ceph) }, }) diff --git a/pkg/cmd/roachtest/tests/s3_microceph.go b/pkg/cmd/roachtest/tests/s3_microceph.go index 6f9850fb2f8b..2f30d3e5bc07 100644 --- a/pkg/cmd/roachtest/tests/s3_microceph.go +++ b/pkg/cmd/roachtest/tests/s3_microceph.go @@ -26,7 +26,8 @@ import ( const cephDisksScript = ` #!/bin/bash for l in a b c; do - loop_file="$(sudo mktemp -p /mnt/data1 XXXX.img)" + mkdir -p /mnt/data1/disks + loop_file="$(sudo mktemp -p /mnt/data1/disks XXXX.img)" sudo truncate -s 4G "${loop_file}" loop_dev="$(sudo losetup --show -f "${loop_file}")" # the block-devices plug doesn't allow accessing /dev/loopX @@ -37,6 +38,17 @@ for l in a b c; do sudo microceph disk add --wipe "/dev/sdi${l}" done` +// cephCleanup remove microceph and the loop devices. +const cephCleanup = ` +#!/bin/bash +sudo microceph disable rgw +sudo snap remove microceph --purge +for l in a b c; do + sudo rm -f /dev/sdi${l} +done +sudo rm -rf /mnt/data1/disks +` + const s3cmdSsl = `sudo s3cmd --host localhost --host-bucket="localhost/%%(bucket)" \ --access_key=%s --secret_key=%s --ca-certs=./certs/ca.crt %s` @@ -81,6 +93,13 @@ func (m cephManager) getBackupURI(ctx context.Context, dest string) (string, err return uri, nil } +func (m cephManager) cleanup(ctx context.Context) { + tmpDir := "/tmp/" + cephCleanupPath := filepath.Join(tmpDir, "cleanup.sh") + m.put(ctx, cephCleanup, cephCleanupPath) + m.run(ctx, "removing ceph", cephCleanupPath, tmpDir) +} + // install a single node microCeph cluster. // See https://canonical-microceph.readthedocs-hosted.com/en/squid-stable/how-to/single-node/ // It is fatal on errors. From afacb22f845964fdf76e4cdea37d6d76a49bd024 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Mon, 13 Jan 2025 09:51:19 -0500 Subject: [PATCH 046/126] backup: skip TestBackupRestoreChecksum under deadlock Fixes #138268 Release note: none --- pkg/backup/backup_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/backup/backup_test.go b/pkg/backup/backup_test.go index d3cd673c8dee..1ff45dcab064 100644 --- a/pkg/backup/backup_test.go +++ b/pkg/backup/backup_test.go @@ -3963,6 +3963,9 @@ func TestBackupRestoreChecksum(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + // Test Server too slow under deadlock. + skip.UnderDeadlock(t) + const numAccounts = 1000 _, sqlDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() From 9a261a669ca969557531061467539b202ffb08df Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 13 Jan 2025 15:55:01 +0100 Subject: [PATCH 047/126] roachtest: prepare sysbench through haproxy Pointing `sysbench prepare` at n1 reliably overloads n1. At time of writing, it will reliably spend the first few minutes of `sysbench run` receiving snapshots. In other words, the benchmark results don't represent actual performance. I did some archaeology to determine why we were bypassing haproxy in the first place, and it turns out that it's because in 2018[^1] the index creations triggered by `prepare` would reliably overwhelm at least one of the nodes, which in turn would prompt `haproxy` to tear down the connections. `sysbench` would react to this by segfaulting; talking directly to n1 was a work-around. Now that seven years have passed and we have introduced various defenses via admission control, let's try to use haproxy again during bulk loading. Ideally, we end up with much more stable benchmark results that represent CockroachDB's actual performance under this workload. [^1]: https://github.com/cockroachdb/cockroach/issues/32738 Epic: CRDB-42584 Release note: None --- pkg/cmd/roachtest/tests/sysbench.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/sysbench.go b/pkg/cmd/roachtest/tests/sysbench.go index 6b46086d5f1d..57de742da28a 100644 --- a/pkg/cmd/roachtest/tests/sysbench.go +++ b/pkg/cmd/roachtest/tests/sysbench.go @@ -185,7 +185,7 @@ func runSysbench(ctx context.Context, t test.Test, c cluster.Cluster, opts sysbe var start time.Time runWorkload := func(ctx context.Context) error { t.Status("preparing workload") - c.Run(ctx, option.WithNodes(c.WorkloadNode()), opts.cmd(false /* haproxy */)+" prepare") + c.Run(ctx, option.WithNodes(c.WorkloadNode()), opts.cmd(true /* haproxy */)+" prepare") t.Status("running workload") cmd := opts.cmd(useHAProxy /* haproxy */) + " run" From 5f7e1446c73877e5842f74ffea7d645431cfc66f Mon Sep 17 00:00:00 2001 From: Annie Pompa Date: Fri, 10 Jan 2025 12:59:37 -0500 Subject: [PATCH 048/126] sql: fix typo in metric description Epic: none Release note: None --- docs/generated/metrics/metrics.html | 2 +- pkg/sql/schema_changer_metrics.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 24e113d71886..38618b673d5e 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -1800,7 +1800,7 @@ APPLICATIONsql.savepoint.started.countNumber of SQL SAVEPOINT statements startedSQL StatementsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.savepoint.started.count.internalNumber of SQL SAVEPOINT statements started (internal queries)SQL Internal StatementsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.schema.invalid_objectsGauge of detected invalid objects within the system.descriptor table (measured by querying crdb_internal.invalid_objects)ObjectsGAUGECOUNTAVGNONE -APPLICATIONsql.schema_changer.object_countCouner of the number of objects in the clusterObjectsGAUGECOUNTAVGNONE +APPLICATIONsql.schema_changer.object_countCounter of the number of objects in the clusterObjectsGAUGECOUNTAVGNONE APPLICATIONsql.select.countNumber of SQL SELECT statements successfully executedSQL StatementsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.select.count.internalNumber of SQL SELECT statements successfully executed (internal queries)SQL Internal StatementsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE APPLICATIONsql.select.started.countNumber of SQL SELECT statements startedSQL StatementsCOUNTERCOUNTAVGNON_NEGATIVE_DERIVATIVE diff --git a/pkg/sql/schema_changer_metrics.go b/pkg/sql/schema_changer_metrics.go index ee137eae0984..d37a1237a409 100644 --- a/pkg/sql/schema_changer_metrics.go +++ b/pkg/sql/schema_changer_metrics.go @@ -16,7 +16,7 @@ import ( var ( metaObjects = metric.Metadata{ Name: "sql.schema_changer.object_count", - Help: "Couner of the number of objects in the cluster", + Help: "Counter of the number of objects in the cluster", Measurement: "Objects", Unit: metric.Unit_COUNT, } From 917b53f9f4bcad346f212a6552e08f6c2082998e Mon Sep 17 00:00:00 2001 From: Chandra Thumuluru Date: Fri, 3 Jan 2025 04:34:23 +0000 Subject: [PATCH 049/126] all: add drpc dependency `drpc` ([github.com/storj/drpc](https://github.com/storj/drpc)) is a lightweight, drop-in replacement to gRPC. Based on the initial benchmark results from the Perf Efficiency team using a simple ping service ([github.com/cockroachlabs/perf-efficiency-team/tree/main/rpctoy](https://github.com/cockroachlabs/perf-efficiency-team/tree/main/rpctoy)), switching from gRPC to drpc offers significant performance gains. This commit introduces a dependency on drpc to facilitate faster experimentation and iteration on the prototype. Epic: None Release note: None --- DEPS.bzl | 20 ++++++++++++++++++++ build/bazelutil/distdir_files.bzl | 2 ++ go.mod | 2 ++ go.sum | 4 ++++ 4 files changed, 28 insertions(+) diff --git a/DEPS.bzl b/DEPS.bzl index 2a7a047cd108..ab45420a1e4e 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -9222,6 +9222,16 @@ def go_deps(): "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/zeebo/assert/com_github_zeebo_assert-v1.3.0.zip", ], ) + go_repository( + name = "com_github_zeebo_errs", + build_file_proto_mode = "disable_global", + importpath = "github.com/zeebo/errs", + sha256 = "d2fa293e275c21bfb413e2968d79036931a55f503d8b62381563ed189b523cd2", + strip_prefix = "github.com/zeebo/errs@v1.2.2", + urls = [ + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/zeebo/errs/com_github_zeebo_errs-v1.2.2.zip", + ], + ) go_repository( name = "com_github_zeebo_xxh3", build_file_proto_mode = "disable_global", @@ -11355,6 +11365,16 @@ def go_deps(): "https://storage.googleapis.com/cockroach-godeps/gomod/rsc.io/sampler/io_rsc_sampler-v1.3.0.zip", ], ) + go_repository( + name = "io_storj_drpc", + build_file_proto_mode = "disable_global", + importpath = "storj.io/drpc", + sha256 = "e297ccead2763d354959a3c04b0c9c27c9c84c99d129f216ec07da663ee0091a", + strip_prefix = "storj.io/drpc@v0.0.34", + urls = [ + "https://storage.googleapis.com/cockroach-godeps/gomod/storj.io/drpc/io_storj_drpc-v0.0.34.zip", + ], + ) go_repository( name = "io_vitess_vitess", build_file_proto_mode = "disable_global", diff --git a/build/bazelutil/distdir_files.bzl b/build/bazelutil/distdir_files.bzl index 0e8cbb7b9221..5658ad5581d1 100644 --- a/build/bazelutil/distdir_files.bzl +++ b/build/bazelutil/distdir_files.bzl @@ -1052,6 +1052,7 @@ DISTDIR_FILES = { "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/z-division/go-zookeeper/com_github_z_division_go_zookeeper-v0.0.0-20190128072838-6d7457066b9b.zip": "b0a67a3bb3cfbb1be18618b84b02588979795966e040f18c5bb4be036888cabd", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/zabawaba99/go-gitignore/com_github_zabawaba99_go_gitignore-v0.0.0-20200117185801-39e6bddfb292.zip": "6c837b93e1c73e53123941c8e866de1deae6b645cc49a7d30d493c146178f8e8", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/zeebo/assert/com_github_zeebo_assert-v1.3.0.zip": "1f01421d74ff37cb8247988155be9e6877d336029bcd887a1d035fd32d7ab6ae", + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/zeebo/errs/com_github_zeebo_errs-v1.2.2.zip": "d2fa293e275c21bfb413e2968d79036931a55f503d8b62381563ed189b523cd2", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/zeebo/xxh3/com_github_zeebo_xxh3-v1.0.2.zip": "190e5ef1f672e9321a1580bdd31c6440fde6044ca8168d2b489cf50cdc4f58a6", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/zenazn/goji/com_github_zenazn_goji-v0.9.0.zip": "0807a255d9d715d18427a6eedd8e4f5a22670b09e5f45fddd229c1ae38da25a9", "https://storage.googleapis.com/cockroach-godeps/gomod/gitlab.com/golang-commonmark/html/com_gitlab_golang_commonmark_html-v0.0.0-20191124015941-a22733972181.zip": "f2ba8985dc9d6be347a17d9200a0be0cee5ab3bce4dc601c0651a77ef2bbffc3", @@ -1190,6 +1191,7 @@ DISTDIR_FILES = { "https://storage.googleapis.com/cockroach-godeps/gomod/sigs.k8s.io/structured-merge-diff/v4/io_k8s_sigs_structured_merge_diff_v4-v4.1.2.zip": "b32af97dadd79179a8f62aaf4ef1e0562e051be77053a60c7a4e724a5cbd00ce", "https://storage.googleapis.com/cockroach-godeps/gomod/sigs.k8s.io/yaml/io_k8s_sigs_yaml-v1.2.0.zip": "55ed08c5df448a033bf7e2c2912d4daa85b856a05c854b0c87ccc85c7f3fbfc7", "https://storage.googleapis.com/cockroach-godeps/gomod/sourcegraph.com/sourcegraph/appdash/com_sourcegraph_sourcegraph_appdash-v0.0.0-20190731080439-ebfcffb1b5c0.zip": "bd2492d9db05362c2fecd0b3d0f6002c89a6d90d678fb93b4158298ab883736f", + "https://storage.googleapis.com/cockroach-godeps/gomod/storj.io/drpc/io_storj_drpc-v0.0.34.zip": "e297ccead2763d354959a3c04b0c9c27c9c84c99d129f216ec07da663ee0091a", "https://storage.googleapis.com/public-bazel-artifacts/bazel/88ef31b429631b787ceb5e4556d773b20ad797c8.zip": "92a89a2bbe6c6db2a8b87da4ce723aff6253656e8417f37e50d362817c39b98b", "https://storage.googleapis.com/public-bazel-artifacts/bazel/bazel-gazelle-v0.39.1.tar.gz": "b760f7fe75173886007f7c2e616a21241208f3d90e8657dc65d36a771e916b6a", "https://storage.googleapis.com/public-bazel-artifacts/bazel/bazel-lib-v1.42.3.tar.gz": "d0529773764ac61184eb3ad3c687fb835df5bee01afedf07f0cf1a45515c96bc", diff --git a/go.mod b/go.mod index bcd5c7337f28..752b5beea53c 100644 --- a/go.mod +++ b/go.mod @@ -41,6 +41,7 @@ require ( google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 google.golang.org/grpc v1.56.3 google.golang.org/protobuf v1.35.1 + storj.io/drpc v0.0.34 ) // If any of the following dependencies get updated as a side-effect @@ -429,6 +430,7 @@ require ( github.com/twpayne/go-kml v1.5.2 // indirect github.com/urfave/cli/v2 v2.3.0 // indirect github.com/yusufpapurcu/wmi v1.2.2 // indirect + github.com/zeebo/errs v1.2.2 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect gitlab.com/golang-commonmark/html v0.0.0-20191124015941-a22733972181 // indirect gitlab.com/golang-commonmark/linkify v0.0.0-20191026162114-a0c2df6c8f82 // indirect diff --git a/go.sum b/go.sum index 0dc48916a769..bf117e5afb38 100644 --- a/go.sum +++ b/go.sum @@ -2363,6 +2363,8 @@ github.com/zabawaba99/go-gitignore v0.0.0-20200117185801-39e6bddfb292 h1:vpcCVk+ github.com/zabawaba99/go-gitignore v0.0.0-20200117185801-39e6bddfb292/go.mod h1:qcqv8IHwbR0JmjY1LZy4PeytlwxDPn1vUkjX7Wq0VaY= github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ= github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= +github.com/zeebo/errs v1.2.2 h1:5NFypMTuSdoySVTqlNs1dEoU21QVamMQJxW/Fii5O7g= +github.com/zeebo/errs v1.2.2/go.mod h1:sgbWHsvVuTPHcqJJGQ1WhI5KbWlHYz+2+2C/LSEtCw4= github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= github.com/zenazn/goji v0.9.0/go.mod h1:7S9M489iMyHBNxwZnk9/EHS098H4/F6TATF2mIxtB1Q= @@ -3323,3 +3325,5 @@ sigs.k8s.io/structured-merge-diff/v4 v4.1.2/go.mod h1:j/nl6xW8vLS49O8YvXW1ocPhZa sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= +storj.io/drpc v0.0.34 h1:q9zlQKfJ5A7x8NQNFk8x7eKUF78FMhmAbZLnFK+og7I= +storj.io/drpc v0.0.34/go.mod h1:Y9LZaa8esL1PW2IDMqJE7CFSNq7d5bQ3RI7mGPtmKMg= From 053101a7f31ea4edfdd9cae2689efa34cf73466a Mon Sep 17 00:00:00 2001 From: Chandra Thumuluru Date: Tue, 7 Jan 2025 11:32:23 +0000 Subject: [PATCH 050/126] rpc: flag to enable kv batch request using drpc DRPC is an experimental feature and is not yet ready for production. It doesn't implement authorization checks. This PR introduces an application setting that can be activated only in test builds or by explicitly setting an environment variable to opt in. Epic: None Release note: None --- pkg/rpc/BUILD.bazel | 3 +++ pkg/rpc/peer_drpc.go | 34 ++++++++++++++++++++++++++++++++++ 2 files changed, 37 insertions(+) create mode 100644 pkg/rpc/peer_drpc.go diff --git a/pkg/rpc/BUILD.bazel b/pkg/rpc/BUILD.bazel index 567eb2ab197b..c79e0b469373 100644 --- a/pkg/rpc/BUILD.bazel +++ b/pkg/rpc/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "keepalive.go", "metrics.go", "peer.go", + "peer_drpc.go", "peer_map.go", "restricted_internal_client.go", "settings.go", @@ -45,6 +46,7 @@ go_library( "//pkg/settings/cluster", "//pkg/ts/tspb", "//pkg/util", + "//pkg/util/buildutil", "//pkg/util/circuit", "//pkg/util/envutil", "//pkg/util/growstack", @@ -53,6 +55,7 @@ go_library( "//pkg/util/log", "//pkg/util/log/logcrash", "//pkg/util/log/severity", + "//pkg/util/metamorphic", "//pkg/util/metric", "//pkg/util/metric/aggmetric", "//pkg/util/netutil", diff --git a/pkg/rpc/peer_drpc.go b/pkg/rpc/peer_drpc.go new file mode 100644 index 000000000000..e1248e1f329d --- /dev/null +++ b/pkg/rpc/peer_drpc.go @@ -0,0 +1,34 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package rpc + +import ( + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" + "github.com/cockroachdb/cockroach/pkg/util/envutil" + "github.com/cockroachdb/errors" +) + +var envExperimentalDRPCEnabled = envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_DRPC_ENABLED", false) + +// ExperimentalDRPCEnabled determines whether a drpc server accepting BatchRequest +// is enabled. This server is experimental and completely unsuitable to production +// usage (for example, does not implement authorization checks). +var ExperimentalDRPCEnabled = settings.RegisterBoolSetting( + settings.ApplicationLevel, + "rpc.experimental_drpc.enabled", + "if true, use drpc to execute Batch RPCs (instead of gRPC)", + envExperimentalDRPCEnabled, + settings.WithValidateBool(func(values *settings.Values, b bool) error { + // drpc support is highly experimental and should not be enabled in production. + // Since authorization is not implemented, we only even host the server if the + // env var is set or it's a CRDB test build. Consequently, these are prereqs + // for setting the cluster setting. + if b && !(envExperimentalDRPCEnabled || buildutil.CrdbTestBuild) { + return errors.New("experimental drpc is not allowed in this environment") + } + return nil + })) From 6a2a8132e1ffff295df5b7c0e839d6a96ee3f7bf Mon Sep 17 00:00:00 2001 From: Chandra Thumuluru Date: Tue, 7 Jan 2025 15:13:42 +0000 Subject: [PATCH 051/126] rpc,server: create drpc server Set up the dRPC server and pass it up the stack using the gRPC server wrapper. Although this approach is somewhat awkward, it is sufficient for the prototype and can be easily refined in subsequent commits. Epic: None Release note: None --- pkg/rpc/BUILD.bazel | 4 +++ pkg/rpc/context.go | 21 ++++++++++++--- pkg/rpc/drpc.go | 54 +++++++++++++++++++++++++++++++++++++++ pkg/server/grpc_server.go | 4 ++- 4 files changed, 78 insertions(+), 5 deletions(-) create mode 100644 pkg/rpc/drpc.go diff --git a/pkg/rpc/BUILD.bazel b/pkg/rpc/BUILD.bazel index c79e0b469373..482199171981 100644 --- a/pkg/rpc/BUILD.bazel +++ b/pkg/rpc/BUILD.bazel @@ -75,6 +75,10 @@ go_library( "@com_github_montanaflynn_stats//:stats", "@com_github_vividcortex_ewma//:ewma", "@io_opentelemetry_go_otel//attribute", + "@io_storj_drpc//drpcmanager", + "@io_storj_drpc//drpcmux", + "@io_storj_drpc//drpcserver", + "@io_storj_drpc//drpcwire", "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//backoff", "@org_golang_google_grpc//codes", diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index d54574e1d3c5..a2bc7cf52af7 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -49,13 +49,15 @@ import ( "google.golang.org/grpc/encoding" "google.golang.org/grpc/metadata" "google.golang.org/grpc/stats" + "storj.io/drpc/drpcmux" + "storj.io/drpc/drpcserver" ) // NewServer sets up an RPC server. Depending on the ServerOptions, the Server // either expects incoming connections from KV nodes, or from tenant SQL // servers. func NewServer(ctx context.Context, rpcCtx *Context, opts ...ServerOption) (*grpc.Server, error) { - srv, _ /* interceptors */, err := NewServerEx(ctx, rpcCtx, opts...) + srv, _, _, err := NewServerEx(ctx, rpcCtx, opts...) return srv, err } @@ -77,13 +79,19 @@ type ClientInterceptorInfo struct { StreamInterceptors []grpc.StreamClientInterceptor } +type DRPCServer struct { + Srv *drpcserver.Server + Mux *drpcmux.Mux + TLSCfg *tls.Config +} + // NewServerEx is like NewServer, but also returns the interceptors that have // been registered with gRPC for the server. These interceptors can be used // manually when bypassing gRPC to call into the server (like the // internalClientAdapter does). func NewServerEx( ctx context.Context, rpcCtx *Context, opts ...ServerOption, -) (s *grpc.Server, sii ServerInterceptorInfo, err error) { +) (s *grpc.Server, d *DRPCServer, sii ServerInterceptorInfo, err error) { var o serverOpts for _, f := range opts { f(&o) @@ -112,7 +120,7 @@ func NewServerEx( if !rpcCtx.ContextOptions.Insecure { tlsConfig, err := rpcCtx.GetServerTLSConfig() if err != nil { - return nil, sii, err + return nil, nil, sii, err } grpcOpts = append(grpcOpts, grpc.Creds(credentials.NewTLS(tlsConfig))) } @@ -184,8 +192,13 @@ func NewServerEx( grpcOpts = append(grpcOpts, grpc.ChainStreamInterceptor(streamInterceptor...)) s = grpc.NewServer(grpcOpts...) + d, err = newDRPCServer(ctx, rpcCtx) + if err != nil { + return nil, nil, ServerInterceptorInfo{}, err + } RegisterHeartbeatServer(s, rpcCtx.NewHeartbeatService()) - return s, ServerInterceptorInfo{ + + return s, d, ServerInterceptorInfo{ UnaryInterceptors: unaryInterceptor, StreamInterceptors: streamInterceptor, }, nil diff --git a/pkg/rpc/drpc.go b/pkg/rpc/drpc.go new file mode 100644 index 000000000000..3fa8ab90a74e --- /dev/null +++ b/pkg/rpc/drpc.go @@ -0,0 +1,54 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package rpc + +import ( + "context" + "math" + + "github.com/cockroachdb/cockroach/pkg/util/log" + "storj.io/drpc/drpcmanager" + "storj.io/drpc/drpcmux" + "storj.io/drpc/drpcserver" + "storj.io/drpc/drpcwire" +) + +func newDRPCServer(ctx context.Context, rpcCtx *Context) (*DRPCServer, error) { + dmux := drpcmux.New() + // NB: any server middleware (server interceptors in gRPC parlance) would go + // here: + // dmux = whateverMiddleware1(dmux) + // dmux = whateverMiddleware2(dmux) + // ... + // + // Each middleware must implement the Handler interface: + // + // HandleRPC(stream Stream, rpc string) error + // + // where Stream + // See here for an example: + // https://github.com/bryk-io/pkg/blob/4da5fbfef47770be376e4022eab5c6c324984bf7/net/drpc/server.go#L91-L101 + + dsrv := drpcserver.NewWithOptions(dmux, drpcserver.Options{ + Log: func(err error) { + log.Warningf(context.Background(), "drpc server error %v", err) + }, + // The reader's max buffer size defaults to 4mb, and if it is exceeded (such + // as happens with AddSSTable) the RPCs fail. + Manager: drpcmanager.Options{Reader: drpcwire.ReaderOptions{MaximumBufferSize: math.MaxInt}}, + }) + + tlsCfg, err := rpcCtx.GetServerTLSConfig() + if err != nil { + return nil, err + } + + return &DRPCServer{ + Srv: dsrv, + Mux: dmux, + TLSCfg: tlsCfg, + }, nil +} diff --git a/pkg/server/grpc_server.go b/pkg/server/grpc_server.go index 7fbc809af58a..862410d1af18 100644 --- a/pkg/server/grpc_server.go +++ b/pkg/server/grpc_server.go @@ -22,6 +22,7 @@ import ( // RPCs. type grpcServer struct { *grpc.Server + drpc *rpc.DRPCServer serverInterceptorsInfo rpc.ServerInterceptorInfo mode serveMode } @@ -29,7 +30,7 @@ type grpcServer struct { func newGRPCServer(ctx context.Context, rpcCtx *rpc.Context) (*grpcServer, error) { s := &grpcServer{} s.mode.set(modeInitializing) - srv, interceptorInfo, err := rpc.NewServerEx( + srv, dsrv, interceptorInfo, err := rpc.NewServerEx( ctx, rpcCtx, rpc.WithInterceptor(func(path string) error { return s.intercept(path) })) @@ -37,6 +38,7 @@ func newGRPCServer(ctx context.Context, rpcCtx *rpc.Context) (*grpcServer, error return nil, err } s.Server = srv + s.drpc = dsrv s.serverInterceptorsInfo = interceptorInfo return s, nil } From 17231906e28a0ee4e374139299941e1ec3257f7f Mon Sep 17 00:00:00 2001 From: Chandra Thumuluru Date: Tue, 7 Jan 2025 17:15:14 +0000 Subject: [PATCH 052/126] server: start drpc listener if enabled Start the DRPC listener to handle server connections. Depending on whether the experiment is enabled, the listener will either be a noop dummy listener or the actual implementation. Epic: None Release note: None --- pkg/server/BUILD.bazel | 4 ++ pkg/server/start_drpc_listener.go | 66 +++++++++++++++++++++++++++++++ pkg/server/start_listen.go | 41 ++++++++++++++++--- 3 files changed, 106 insertions(+), 5 deletions(-) create mode 100644 pkg/server/start_drpc_listener.go diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index b31d88567417..f6af62963269 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -368,6 +368,7 @@ go_library( "@com_github_prometheus_client_model//go", "@com_github_prometheus_common//expfmt", "@in_gopkg_yaml_v2//:yaml_v2", + "@io_storj_drpc//drpcmigrate", "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//metadata", @@ -439,6 +440,7 @@ go_test( "critical_nodes_test.go", "distsql_flows_test.go", "drain_test.go", + "drpc_test.go", "get_local_files_test.go", "graphite_test.go", "grpc_gateway_test.go", @@ -600,6 +602,8 @@ go_test( "@com_github_stretchr_testify//require", "@in_gopkg_yaml_v2//:yaml_v2", "@io_opentelemetry_go_otel//attribute", + "@io_storj_drpc//drpcconn", + "@io_storj_drpc//drpcmigrate", "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//metadata", diff --git a/pkg/server/start_drpc_listener.go b/pkg/server/start_drpc_listener.go new file mode 100644 index 000000000000..96bd999a100c --- /dev/null +++ b/pkg/server/start_drpc_listener.go @@ -0,0 +1,66 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package server + +import ( + "bytes" + "io" + "net" + + "storj.io/drpc/drpcmigrate" +) + +var drpcMatcher = func(reader io.Reader) bool { + buf := make([]byte, len(drpcmigrate.DRPCHeader)) + if _, err := io.ReadFull(reader, buf); err != nil { + return false + } + return bytes.Equal(buf, []byte(drpcmigrate.DRPCHeader)) +} + +type dropDRPCHeaderListener struct { + wrapped net.Listener +} + +func (ln *dropDRPCHeaderListener) Accept() (net.Conn, error) { + conn, err := ln.wrapped.Accept() + if err != nil { + return nil, err + } + buf := make([]byte, len(drpcmigrate.DRPCHeader)) + if _, err := io.ReadFull(conn, buf); err != nil { + return nil, err + } + return conn, nil +} + +func (ln *dropDRPCHeaderListener) Close() error { + return ln.wrapped.Close() +} + +func (ln *dropDRPCHeaderListener) Addr() net.Addr { + return ln.wrapped.Addr() +} + +type noopListener struct{ done chan struct{} } + +func (l *noopListener) Accept() (net.Conn, error) { + <-l.done + return nil, net.ErrClosed +} + +func (l *noopListener) Close() error { + if l.done == nil { + return nil + } + close(l.done) + l.done = nil + return nil +} + +func (l *noopListener) Addr() net.Addr { + return nil +} diff --git a/pkg/server/start_listen.go b/pkg/server/start_listen.go index 456cd7206ba7..59471cefde2b 100644 --- a/pkg/server/start_listen.go +++ b/pkg/server/start_listen.go @@ -7,6 +7,7 @@ package server import ( "context" + "crypto/tls" "io" "net" "sync" @@ -131,21 +132,41 @@ func startListenRPCAndSQL( } } - anyL := m.Match(cmux.Any()) + // Host drpc only if it's _possible_ to turn it on (this requires a test build + // or env var). If the setting _is_ on, then it was overridden in testing and + // we want to host the server too. + hostDRPC := rpc.ExperimentalDRPCEnabled.Validate(nil /* not used */, true) == nil || + rpc.ExperimentalDRPCEnabled.Get(&cfg.Settings.SV) + + // If we're not hosting drpc, make a listener that never accepts anything. + // We will start the dRPC server all the same; it barely consumes any + // resources. + var drpcL net.Listener = &noopListener{make(chan struct{})} + if hostDRPC { + // Throw away the header before passing the conn to the drpc server. This + // would not be required explicitly if we used `drpcmigrate.ListenMux` but + // cmux keeps the prefix. + drpcL = &dropDRPCHeaderListener{wrapped: m.Match(drpcMatcher)} + } + + grpcL := m.Match(cmux.Any()) if serverTestKnobs, ok := cfg.TestingKnobs.Server.(*TestingKnobs); ok { if serverTestKnobs.ContextTestingKnobs.InjectedLatencyOracle != nil { - anyL = rpc.NewDelayingListener(anyL, serverTestKnobs.ContextTestingKnobs.InjectedLatencyEnabled) + grpcL = rpc.NewDelayingListener(grpcL, serverTestKnobs.ContextTestingKnobs.InjectedLatencyEnabled) + drpcL = rpc.NewDelayingListener(drpcL, serverTestKnobs.ContextTestingKnobs.InjectedLatencyEnabled) } } rpcLoopbackL := netutil.NewLoopbackListener(ctx, stopper) sqlLoopbackL := netutil.NewLoopbackListener(ctx, stopper) + drpcCtx, drpcCancel := context.WithCancel(workersCtx) // The remainder shutdown worker. waitForQuiesce := func(context.Context) { <-stopper.ShouldQuiesce() + drpcCancel() // TODO(bdarnell): Do we need to also close the other listeners? - netutil.FatalIfUnexpected(anyL.Close()) + netutil.FatalIfUnexpected(grpcL.Close()) netutil.FatalIfUnexpected(rpcLoopbackL.Close()) netutil.FatalIfUnexpected(sqlLoopbackL.Close()) netutil.FatalIfUnexpected(ln.Close()) @@ -160,12 +181,14 @@ func startListenRPCAndSQL( netutil.FatalIfUnexpected(m.Serve()) }) } + stopper.AddCloser(stop.CloserFn(stopGRPC)) if err := stopper.RunAsyncTask( - workersCtx, "grpc-quiesce", waitForQuiesce, + workersCtx, "grpc-drpc-quiesce", waitForQuiesce, ); err != nil { waitForQuiesce(ctx) stopGRPC() + drpcCancel() return nil, nil, nil, nil, err } stopper.AddCloser(stop.CloserFn(stopGRPC)) @@ -177,7 +200,15 @@ func startListenRPCAndSQL( startRPCServer = func(ctx context.Context) { // Serve the gRPC endpoint. _ = stopper.RunAsyncTask(workersCtx, "serve-grpc", func(context.Context) { - netutil.FatalIfUnexpected(grpc.Serve(anyL)) + netutil.FatalIfUnexpected(grpc.Serve(grpcL)) + }) + _ = stopper.RunAsyncTask(drpcCtx, "serve-drpc", func(ctx context.Context) { + if cfg := grpc.drpc.TLSCfg; cfg != nil { + drpcTLSL := tls.NewListener(drpcL, cfg) + netutil.FatalIfUnexpected(grpc.drpc.Srv.Serve(ctx, drpcTLSL)) + } else { + netutil.FatalIfUnexpected(grpc.drpc.Srv.Serve(ctx, drpcL)) + } }) _ = stopper.RunAsyncTask(workersCtx, "serve-loopback-grpc", func(context.Context) { netutil.FatalIfUnexpected(grpc.Serve(rpcLoopbackL)) From 9369fd0fa9e3720f0bf9205c178f79aad1727bc1 Mon Sep 17 00:00:00 2001 From: Chandra Thumuluru Date: Wed, 8 Jan 2025 03:28:24 +0000 Subject: [PATCH 053/126] kv,server: register batch service with DRPC server Register KV batch and streaming batch methods from internal service with DRPC server. After these server side changes, when hosting DRPC is enabled, clients can connect to KV nodes and make DRPC batch and streaming batch requests. Client changes will be done in a separate commit. Epic: None Release note: None --- pkg/kv/kvpb/BUILD.bazel | 12 +- pkg/kv/kvpb/api_drpc_hacky.go | 207 ++++++++++++++++++++++++++++++++++ pkg/rpc/context_test.go | 8 +- pkg/server/BUILD.bazel | 1 + pkg/server/node.go | 36 +++++- pkg/server/server.go | 3 + 6 files changed, 257 insertions(+), 10 deletions(-) create mode 100644 pkg/kv/kvpb/api_drpc_hacky.go diff --git a/pkg/kv/kvpb/BUILD.bazel b/pkg/kv/kvpb/BUILD.bazel index 9f4d3bb90f64..c1da83856de8 100644 --- a/pkg/kv/kvpb/BUILD.bazel +++ b/pkg/kv/kvpb/BUILD.bazel @@ -8,17 +8,21 @@ load(":gen.bzl", "batch_gen") go_library( name = "kvpb", srcs = [ + ":gen-batch-generated", # keep + ":gen-errordetailtype-stringer", # keep + ":gen-method-stringer", # keep "ambiguous_result_error.go", "api.go", + # DRPC protobuf file (api_drpc.pb.go) is currently generated manually. + # TODO (chandrat): Remove this once DRPC protobuf generation is + # integrated into the build process. + "api_drpc_hacky.go", "batch.go", "data.go", "errors.go", "method.go", "node_decommissioned_error.go", "replica_unavailable_error.go", - ":gen-batch-generated", # keep - ":gen-errordetailtype-stringer", # keep - ":gen-method-stringer", # keep ], embed = [":kvpb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvpb", @@ -46,6 +50,8 @@ go_library( "@com_github_gogo_protobuf//types", "@com_github_gogo_status//:status", "@com_github_golang_mock//gomock", # keep + "@io_storj_drpc//:drpc", + "@io_storj_drpc//drpcerr", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//metadata", # keep ], diff --git a/pkg/kv/kvpb/api_drpc_hacky.go b/pkg/kv/kvpb/api_drpc_hacky.go new file mode 100644 index 000000000000..1dc6b640ee39 --- /dev/null +++ b/pkg/kv/kvpb/api_drpc_hacky.go @@ -0,0 +1,207 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +// This file was manually generated with the DRPC protogen plugin using a dummy +// `api.proto` that includes a subset of relevant service methods. +// +// For instance, to generate this file, following proto file was used: +// +// -- api.proto -- begin -- +// syntax = "proto3"; +// package cockroach.kv.kvpb; +// option go_package = "github.com/cockroachdb/cockroach/pkg/kv/kvpb"; +// service Batch { +// rpc Batch (BatchRequest) returns (BatchResponse) {} +// rpc BatchStream (stream BatchRequest) returns (stream BatchResponse) {} +// } +// message BatchRequest{} +// message BatchResponse{} +// -- api.proto -- end -- +// +// NB: The use of empty BatchRequest and BatchResponse messages is a deliberate +// decision to avoid dependencies. +// +// +// To generate this file using DRPC protogen plugin from the dummy `api.proto` +// defined above, use the following command: +// +// ``` +// protoc --gogo_out=paths=source_relative:. \ +// --go-drpc_out=paths=source_relative,protolib=github.com/gogo/protobuf:. \ +// api.proto +// ``` +// +// NB: Make sure you have `protoc` installed and `protoc-gen-gogoroach` is +// built from $COCKROACH_SRC/pkg/cmd/protoc-gen-gogoroach. +// +// This code-gen should be automated as part of productionizing drpc. + +package kvpb + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" + "storj.io/drpc" + "storj.io/drpc/drpcerr" +) + +type drpcEncoding_File_api_proto struct{} + +func (drpcEncoding_File_api_proto) Marshal(msg drpc.Message) ([]byte, error) { + return protoutil.Marshal(msg.(protoutil.Message)) +} + +func (drpcEncoding_File_api_proto) Unmarshal(buf []byte, msg drpc.Message) error { + return protoutil.Unmarshal(buf, msg.(protoutil.Message)) +} + +type DRPCBatchClient interface { + DRPCConn() drpc.Conn + + Batch(ctx context.Context, in *BatchRequest) (*BatchResponse, error) + BatchStream(ctx context.Context) (DRPCBatch_BatchStreamClient, error) +} + +type drpcBatchClient struct { + cc drpc.Conn +} + +func NewDRPCBatchClient(cc drpc.Conn) DRPCBatchClient { + return &drpcBatchClient{cc} +} + +func (c *drpcBatchClient) DRPCConn() drpc.Conn { return c.cc } + +func (c *drpcBatchClient) Batch(ctx context.Context, in *BatchRequest) (*BatchResponse, error) { + out := new(BatchResponse) + err := c.cc.Invoke(ctx, "/cockroach.kv.kvpb.Batch/Batch", drpcEncoding_File_api_proto{}, in, out) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *drpcBatchClient) BatchStream(ctx context.Context) (DRPCBatch_BatchStreamClient, error) { + stream, err := c.cc.NewStream(ctx, "/cockroach.kv.kvpb.Batch/BatchStream", drpcEncoding_File_api_proto{}) + if err != nil { + return nil, err + } + x := &drpcBatch_BatchStreamClient{stream} + return x, nil +} + +type DRPCBatch_BatchStreamClient interface { + drpc.Stream + Send(*BatchRequest) error + Recv() (*BatchResponse, error) +} + +type drpcBatch_BatchStreamClient struct { + drpc.Stream +} + +func (x *drpcBatch_BatchStreamClient) GetStream() drpc.Stream { + return x.Stream +} + +func (x *drpcBatch_BatchStreamClient) Send(m *BatchRequest) error { + return x.MsgSend(m, drpcEncoding_File_api_proto{}) +} + +func (x *drpcBatch_BatchStreamClient) Recv() (*BatchResponse, error) { + m := new(BatchResponse) + if err := x.MsgRecv(m, drpcEncoding_File_api_proto{}); err != nil { + return nil, err + } + return m, nil +} + +func (x *drpcBatch_BatchStreamClient) RecvMsg(m *BatchResponse) error { + return x.MsgRecv(m, drpcEncoding_File_api_proto{}) +} + +type DRPCBatchServer interface { + Batch(context.Context, *BatchRequest) (*BatchResponse, error) + BatchStream(DRPCBatch_BatchStreamStream) error +} + +type DRPCBatchUnimplementedServer struct{} + +func (s *DRPCBatchUnimplementedServer) Batch( + context.Context, *BatchRequest, +) (*BatchResponse, error) { + return nil, drpcerr.WithCode(errors.New("Unimplemented"), drpcerr.Unimplemented) +} + +func (s *DRPCBatchUnimplementedServer) BatchStream(DRPCBatch_BatchStreamStream) error { + return drpcerr.WithCode(errors.New("Unimplemented"), drpcerr.Unimplemented) +} + +type DRPCBatchDescription struct{} + +func (DRPCBatchDescription) NumMethods() int { return 2 } + +func (DRPCBatchDescription) Method( + n int, +) (string, drpc.Encoding, drpc.Receiver, interface{}, bool) { + switch n { + case 0: + return "/cockroach.kv.kvpb.Batch/Batch", drpcEncoding_File_api_proto{}, + func(srv interface{}, ctx context.Context, in1, in2 interface{}) (drpc.Message, error) { + return srv.(DRPCBatchServer). + Batch( + ctx, + in1.(*BatchRequest), + ) + }, DRPCBatchServer.Batch, true + case 1: + return "/cockroach.kv.kvpb.Batch/BatchStream", drpcEncoding_File_api_proto{}, + func(srv interface{}, ctx context.Context, in1, in2 interface{}) (drpc.Message, error) { + return nil, srv.(DRPCBatchServer). + BatchStream( + &drpcBatch_BatchStreamStream{in1.(drpc.Stream)}, + ) + }, DRPCBatchServer.BatchStream, true + default: + return "", nil, nil, nil, false + } +} + +func DRPCRegisterBatch(mux drpc.Mux, impl DRPCBatchServer) error { + return mux.Register(impl, DRPCBatchDescription{}) +} + +type DRPCBatch_BatchStream interface { + drpc.Stream + SendAndClose(*BatchResponse) error +} + +type DRPCBatch_BatchStreamStream interface { + drpc.Stream + Send(*BatchResponse) error + Recv() (*BatchRequest, error) +} + +type drpcBatch_BatchStreamStream struct { + drpc.Stream +} + +func (x *drpcBatch_BatchStreamStream) Send(m *BatchResponse) error { + return x.MsgSend(m, drpcEncoding_File_api_proto{}) +} + +func (x *drpcBatch_BatchStreamStream) Recv() (*BatchRequest, error) { + m := new(BatchRequest) + if err := x.MsgRecv(m, drpcEncoding_File_api_proto{}); err != nil { + return nil, err + } + return m, nil +} + +func (x *drpcBatch_BatchStreamStream) RecvMsg(m *BatchRequest) error { + return x.MsgRecv(m, drpcEncoding_File_api_proto{}) +} diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index d81a8ff5f6d8..7a01440d8f20 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -434,7 +434,7 @@ func TestInternalClientAdapterRunsInterceptors(t *testing.T) { serverCtx.AdvertiseAddr = "127.0.0.1:8888" serverCtx.NodeID.Set(context.Background(), 1) - _ /* server */, serverInterceptors, err := NewServerEx(ctx, serverCtx) + _ /* gRPC server */, _ /* drpc server */, serverInterceptors, err := NewServerEx(ctx, serverCtx) require.NoError(t, err) // Pile on one more interceptor to make sure it's called. @@ -535,7 +535,7 @@ func TestInternalClientAdapterWithClientStreamInterceptors(t *testing.T) { serverCtx.AdvertiseAddr = "127.0.0.1:8888" serverCtx.NodeID.Set(context.Background(), 1) - _ /* server */, serverInterceptors, err := NewServerEx(ctx, serverCtx) + _ /* gRPC server */, _ /* drpc server */, serverInterceptors, err := NewServerEx(ctx, serverCtx) require.NoError(t, err) var clientInterceptors ClientInterceptorInfo var s *testClientStream @@ -598,7 +598,7 @@ func TestInternalClientAdapterWithServerStreamInterceptors(t *testing.T) { serverCtx.AdvertiseAddr = "127.0.0.1:8888" serverCtx.NodeID.Set(context.Background(), 1) - _ /* server */, serverInterceptors, err := NewServerEx(ctx, serverCtx) + _ /* gRPC server */, _ /* drpc server */, serverInterceptors, err := NewServerEx(ctx, serverCtx) require.NoError(t, err) const int1Name = "interceptor 1" @@ -736,7 +736,7 @@ func BenchmarkInternalClientAdapter(b *testing.B) { serverCtx.AdvertiseAddr = "127.0.0.1:8888" serverCtx.NodeID.Set(context.Background(), 1) - _, interceptors, err := NewServerEx(ctx, serverCtx) + _ /* gRPC server */, _ /* drpc server */, interceptors, err := NewServerEx(ctx, serverCtx) require.NoError(b, err) internal := &internalServer{} diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index f6af62963269..a9879a6b443b 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -64,6 +64,7 @@ go_library( "span_download.go", "span_stats_server.go", "sql_stats.go", + "start_drpc_listener.go", "start_listen.go", "statement_details.go", "statement_diagnostics_requests.go", diff --git a/pkg/server/node.go b/pkg/server/node.go index b319a9892a3a..3dad84c9670d 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -1875,6 +1875,18 @@ func (n *Node) Batch(ctx context.Context, args *kvpb.BatchRequest) (*kvpb.BatchR // BatchStream implements the kvpb.InternalServer interface. func (n *Node) BatchStream(stream kvpb.Internal_BatchStreamServer) error { + return n.batchStreamImpl(stream, func(ba *kvpb.BatchRequest) error { + return stream.RecvMsg(ba) + }) +} + +func (n *Node) batchStreamImpl( + stream interface { + Context() context.Context + Send(response *kvpb.BatchResponse) error + }, + recvMsg func(*kvpb.BatchRequest) error, +) error { ctx := stream.Context() for { argsAlloc := new(struct { @@ -1884,10 +1896,8 @@ func (n *Node) BatchStream(stream kvpb.Internal_BatchStreamServer) error { args := &argsAlloc.args args.Requests = argsAlloc.reqs[:0] - err := stream.RecvMsg(args) + err := recvMsg(args) if err != nil { - // From grpc.ServerStream.Recv: - // > It returns io.EOF when the client has performed a CloseSend. if errors.Is(err, io.EOF) { return nil } @@ -1905,6 +1915,26 @@ func (n *Node) BatchStream(stream kvpb.Internal_BatchStreamServer) error { } } +func (n *Node) AsDRPCBatchServer() kvpb.DRPCBatchServer { + return (*drpcNode)(n) +} + +type drpcNode Node + +func (n *drpcNode) Batch( + ctx context.Context, request *kvpb.BatchRequest, +) (*kvpb.BatchResponse, error) { + return (*Node)(n).Batch(ctx, request) +} + +func (n *drpcNode) BatchStream(stream kvpb.DRPCBatch_BatchStreamStream) error { + return (*Node)(n).batchStreamImpl(stream, func(ba *kvpb.BatchRequest) error { + return stream.(interface { + RecvMsg(request *kvpb.BatchRequest) error + }).RecvMsg(ba) + }) +} + // spanForRequest is the retval of setupSpanForIncomingRPC. It groups together a // few variables needed when finishing an RPC's span. // diff --git a/pkg/server/server.go b/pkg/server/server.go index c462243e4ebb..0666cb3c9fb5 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -957,6 +957,9 @@ func NewServer(cfg Config, stopper *stop.Stopper) (serverctl.ServerStartupInterf cfg.LicenseEnforcer, ) kvpb.RegisterInternalServer(grpcServer.Server, node) + if err := kvpb.DRPCRegisterBatch(grpcServer.drpc.Mux, node.AsDRPCBatchServer()); err != nil { + return nil, err + } kvserver.RegisterPerReplicaServer(grpcServer.Server, node.perReplicaServer) kvserver.RegisterPerStoreServer(grpcServer.Server, node.perReplicaServer) ctpb.RegisterSideTransportServer(grpcServer.Server, ctReceiver) From b0ae3171018e556096f005af3ff4930f5ac8d143 Mon Sep 17 00:00:00 2001 From: Chandra Thumuluru Date: Wed, 8 Jan 2025 05:27:44 +0000 Subject: [PATCH 054/126] rpc, kv: use drpc client for batch requests This commit allows `*rpc.Connection` to also maintain a `drpcpool.Conn`. When drpc is enabled, clients have a choice to either use gRPC or drpc. Epic: None Release note: None --- pkg/kv/kvserver/loqrecovery/server.go | 4 +- pkg/rpc/BUILD.bazel | 7 ++- pkg/rpc/connection.go | 56 +++++++++++++++------ pkg/rpc/drpc.go | 71 +++++++++++++++++++++++++++ pkg/rpc/nodedialer/BUILD.bazel | 6 ++- pkg/rpc/nodedialer/nodedialer.go | 41 +++++++++++----- pkg/rpc/nodedialer/nodedialer_drpc.go | 35 +++++++++++++ pkg/rpc/peer.go | 21 ++++++-- pkg/rpc/stream_pool.go | 10 ++++ 9 files changed, 217 insertions(+), 34 deletions(-) create mode 100644 pkg/rpc/nodedialer/nodedialer_drpc.go diff --git a/pkg/kv/kvserver/loqrecovery/server.go b/pkg/kv/kvserver/loqrecovery/server.go index 9898724a3943..cf8adfa23a4c 100644 --- a/pkg/kv/kvserver/loqrecovery/server.go +++ b/pkg/kv/kvserver/loqrecovery/server.go @@ -750,7 +750,7 @@ func visitNodeWithRetry( // Note that we use ConnectNoBreaker here to avoid any race with probe // running on current node and target node restarting. Errors from circuit // breaker probes could confuse us and present node as unavailable. - conn, err = rpcCtx.GRPCDialNode(addr.String(), node.NodeID, node.Locality, rpc.DefaultClass).ConnectNoBreaker(ctx) + conn, _, err = rpcCtx.GRPCDialNode(addr.String(), node.NodeID, node.Locality, rpc.DefaultClass).ConnectNoBreaker(ctx) // Nodes would contain dead nodes that we don't need to visit. We can skip // them and let caller handle incomplete info. if err != nil { @@ -803,7 +803,7 @@ func makeVisitNode(g *gossip.Gossip, loc roachpb.Locality, rpcCtx *rpc.Context) // Note that we use ConnectNoBreaker here to avoid any race with probe // running on current node and target node restarting. Errors from circuit // breaker probes could confuse us and present node as unavailable. - conn, err = rpcCtx.GRPCDialNode(addr.String(), node.NodeID, node.Locality, rpc.DefaultClass).ConnectNoBreaker(ctx) + conn, _, err = rpcCtx.GRPCDialNode(addr.String(), node.NodeID, node.Locality, rpc.DefaultClass).ConnectNoBreaker(ctx) if err != nil { if grpcutil.IsClosedConnection(err) { log.Infof(ctx, "can't dial node n%d because connection is permanently closed: %s", diff --git a/pkg/rpc/BUILD.bazel b/pkg/rpc/BUILD.bazel index 482199171981..3f3ca878487e 100644 --- a/pkg/rpc/BUILD.bazel +++ b/pkg/rpc/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "connection_class.go", "context.go", "context_testutils.go", + "drpc.go", "errors.go", "heartbeat.go", "keepalive.go", @@ -55,7 +56,6 @@ go_library( "//pkg/util/log", "//pkg/util/log/logcrash", "//pkg/util/log/severity", - "//pkg/util/metamorphic", "//pkg/util/metric", "//pkg/util/metric/aggmetric", "//pkg/util/netutil", @@ -75,9 +75,14 @@ go_library( "@com_github_montanaflynn_stats//:stats", "@com_github_vividcortex_ewma//:ewma", "@io_opentelemetry_go_otel//attribute", + "@io_storj_drpc//:drpc", + "@io_storj_drpc//drpcconn", "@io_storj_drpc//drpcmanager", + "@io_storj_drpc//drpcmigrate", "@io_storj_drpc//drpcmux", + "@io_storj_drpc//drpcpool", "@io_storj_drpc//drpcserver", + "@io_storj_drpc//drpcstream", "@io_storj_drpc//drpcwire", "@org_golang_google_grpc//:grpc", "@org_golang_google_grpc//backoff", diff --git a/pkg/rpc/connection.go b/pkg/rpc/connection.go index 2adfd4560370..232df445aa80 100644 --- a/pkg/rpc/connection.go +++ b/pkg/rpc/connection.go @@ -11,6 +11,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/circuit" "github.com/cockroachdb/errors" "google.golang.org/grpc" + "storj.io/drpc/drpcpool" ) // Connection is a wrapper around grpc.ClientConn. It prevents the underlying @@ -39,7 +40,8 @@ type Connection struct { // RPCs. // // The pool is only initialized once the ClientConn is resolved. - batchStreamPool BatchStreamPool + batchStreamPool BatchStreamPool + drpcBatchStreamPool DRPCBatchStreamPool } // newConnectionToNodeID makes a Connection for the given node, class, and nontrivial Signal @@ -53,7 +55,8 @@ func newConnectionToNodeID( connFuture: connFuture{ ready: make(chan struct{}), }, - batchStreamPool: makeStreamPool(opts.Stopper, newBatchStream), + batchStreamPool: makeStreamPool(opts.Stopper, newBatchStream), + drpcBatchStreamPool: makeStreamPool(opts.Stopper, newDRPCBatchStream), } return c } @@ -65,14 +68,14 @@ func newConnectionToNodeID( // block but fall back to defErr in this case. func (c *Connection) waitOrDefault( ctx context.Context, defErr error, sig circuit.Signal, -) (*grpc.ClientConn, error) { +) (*grpc.ClientConn, drpcpool.Conn, error) { // Check the circuit breaker first. If it is already tripped now, we // want it to take precedence over connFuture below (which is closed in // the common case of a connection going bad after having been healthy // for a while). select { case <-sig.C(): - return nil, sig.Err() + return nil, nil, sig.Err() default: } @@ -83,26 +86,26 @@ func (c *Connection) waitOrDefault( select { case <-c.connFuture.C(): case <-sig.C(): - return nil, sig.Err() + return nil, nil, sig.Err() case <-ctx.Done(): - return nil, errors.Wrapf(ctx.Err(), "while connecting to n%d at %s", c.k.NodeID, c.k.TargetAddr) + return nil, nil, errors.Wrapf(ctx.Err(), "while connecting to n%d at %s", c.k.NodeID, c.k.TargetAddr) } } else { select { case <-c.connFuture.C(): case <-sig.C(): - return nil, sig.Err() + return nil, nil, sig.Err() case <-ctx.Done(): - return nil, errors.Wrapf(ctx.Err(), "while connecting to n%d at %s", c.k.NodeID, c.k.TargetAddr) + return nil, nil, errors.Wrapf(ctx.Err(), "while connecting to n%d at %s", c.k.NodeID, c.k.TargetAddr) default: - return nil, defErr + return nil, nil, defErr } } // Done waiting, c.connFuture has resolved, return the result. Note that this // conn could be unhealthy (or there may not even be a conn, i.e. Err() != // nil), if that's what the caller wanted (ConnectNoBreaker). - return c.connFuture.Conn(), c.connFuture.Err() + return c.connFuture.Conn(), c.connFuture.DRPCConn(), c.connFuture.Err() } // Connect returns the underlying grpc.ClientConn after it has been validated, @@ -112,6 +115,13 @@ func (c *Connection) waitOrDefault( // an error. In rare cases, this behavior is undesired and ConnectNoBreaker may // be used instead. func (c *Connection) Connect(ctx context.Context) (*grpc.ClientConn, error) { + cc, _, err := c.waitOrDefault(ctx, nil /* defErr */, c.breakerSignalFn()) + return cc, err +} + +// ConnectEx is similar to Connect but it addition to gRPC connection, it also +// returns underlying drpc connection after it has been validated. +func (c *Connection) ConnectEx(ctx context.Context) (*grpc.ClientConn, drpcpool.Conn, error) { return c.waitOrDefault(ctx, nil /* defErr */, c.breakerSignalFn()) } @@ -133,7 +143,9 @@ func (s *neverTripSignal) IsTripped() bool { // that it will latch onto (or start) an existing connection attempt even if // previous attempts have not succeeded. This may be preferable to Connect // if the caller is already certain that a peer is available. -func (c *Connection) ConnectNoBreaker(ctx context.Context) (*grpc.ClientConn, error) { +func (c *Connection) ConnectNoBreaker( + ctx context.Context, +) (*grpc.ClientConn, drpcpool.Conn, error) { // For ConnectNoBreaker we don't use the default Signal but pass a dummy one // that never trips. (The probe tears down the Conn on quiesce so we don't rely // on the Signal for that). @@ -157,7 +169,7 @@ func (c *Connection) ConnectNoBreaker(ctx context.Context) (*grpc.ClientConn, er // latest heartbeat. Returns ErrNotHeartbeated if the peer was just contacted for // the first time and the first heartbeat has not occurred yet. func (c *Connection) Health() error { - _, err := c.waitOrDefault(context.Background(), ErrNotHeartbeated, c.breakerSignalFn()) + _, _, err := c.waitOrDefault(context.Background(), ErrNotHeartbeated, c.breakerSignalFn()) return err } @@ -172,9 +184,17 @@ func (c *Connection) BatchStreamPool() *BatchStreamPool { return &c.batchStreamPool } +func (c *Connection) DRPCBatchStreamPool() *DRPCBatchStreamPool { + if !c.connFuture.Resolved() { + panic("DRPCBatchStreamPool called on unresolved connection") + } + return &c.drpcBatchStreamPool +} + type connFuture struct { ready chan struct{} cc *grpc.ClientConn + dc drpcpool.Conn err error } @@ -201,6 +221,14 @@ func (s *connFuture) Conn() *grpc.ClientConn { return s.cc } +// DRPCConn must only be called after C() has been closed. +func (s *connFuture) DRPCConn() drpcpool.Conn { + if s.err != nil { + return nil + } + return s.dc +} + func (s *connFuture) Resolved() bool { select { case <-s.ready: @@ -212,12 +240,12 @@ func (s *connFuture) Resolved() bool { // Resolve is idempotent. Only the first call has any effect. // Not thread safe. -func (s *connFuture) Resolve(cc *grpc.ClientConn, err error) { +func (s *connFuture) Resolve(cc *grpc.ClientConn, dc drpcpool.Conn, err error) { select { case <-s.ready: // Already resolved, noop. default: - s.cc, s.err = cc, err + s.cc, s.dc, s.err = cc, dc, err close(s.ready) } } diff --git a/pkg/rpc/drpc.go b/pkg/rpc/drpc.go index 3fa8ab90a74e..4af695162801 100644 --- a/pkg/rpc/drpc.go +++ b/pkg/rpc/drpc.go @@ -7,12 +7,17 @@ package rpc import ( "context" + "crypto/tls" "math" "github.com/cockroachdb/cockroach/pkg/util/log" + "storj.io/drpc/drpcconn" "storj.io/drpc/drpcmanager" + "storj.io/drpc/drpcmigrate" "storj.io/drpc/drpcmux" + "storj.io/drpc/drpcpool" "storj.io/drpc/drpcserver" + "storj.io/drpc/drpcstream" "storj.io/drpc/drpcwire" ) @@ -52,3 +57,69 @@ func newDRPCServer(ctx context.Context, rpcCtx *Context) (*DRPCServer, error) { TLSCfg: tlsCfg, }, nil } + +func dialDRPC(rpcCtx *Context) func(ctx context.Context, target string) (drpcpool.Conn, error) { + return func(ctx context.Context, target string) (drpcpool.Conn, error) { + // TODO(server): could use connection class instead of empty key here. + pool := drpcpool.New[struct{}, drpcpool.Conn](drpcpool.Options{}) + pooledConn := pool.Get(ctx /* unused */, struct{}{}, func(ctx context.Context, + _ struct{}) (drpcpool.Conn, error) { + + netConn, err := drpcmigrate.DialWithHeader(ctx, "tcp", target, drpcmigrate.DRPCHeader) + if err != nil { + return nil, err + } + + opts := drpcconn.Options{ + Manager: drpcmanager.Options{ + Reader: drpcwire.ReaderOptions{ + MaximumBufferSize: math.MaxInt, + }, + Stream: drpcstream.Options{ + MaximumBufferSize: 0, // unlimited + }, + }, + } + var conn *drpcconn.Conn + if rpcCtx.ContextOptions.Insecure { + conn = drpcconn.NewWithOptions(netConn, opts) + } else { + tlsConfig, err := rpcCtx.GetClientTLSConfig() + if err != nil { + return nil, err + } + // TODO(server): at least with testing certs, we get: manager closed: + // tls: either ServerName or InsecureSkipVerify must be specified in + // the tls.Config from drpcmanager.(*Manager).manageReader:234 + // + // This is possibly avoided in gRPC by setting ServerName in + // (*tlsCreds).ClientHandshake. + tlsConfig = tlsConfig.Clone() + tlsConfig.InsecureSkipVerify = true // HACK + tlsConn := tls.Client(netConn, tlsConfig) + conn = drpcconn.NewWithOptions(tlsConn, opts) + } + + return conn, nil + }) + // `pooledConn.Close` doesn't tear down any of the underlying TCP + // connections but simply marks the pooledConn handle as returning + // errors. When we "close" this conn, we want to tear down all of + // the connections in the pool (in effect mirroring the behavior of + // gRPC where a single conn is shared). + return &closeEntirePoolConn{ + Conn: pooledConn, + pool: pool, + }, nil + } +} + +type closeEntirePoolConn struct { + drpcpool.Conn + pool *drpcpool.Pool[struct{}, drpcpool.Conn] +} + +func (c *closeEntirePoolConn) Close() error { + _ = c.Conn.Close() + return c.pool.Close() +} diff --git a/pkg/rpc/nodedialer/BUILD.bazel b/pkg/rpc/nodedialer/BUILD.bazel index b405795e4d32..76e785a8bab0 100644 --- a/pkg/rpc/nodedialer/BUILD.bazel +++ b/pkg/rpc/nodedialer/BUILD.bazel @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "nodedialer", - srcs = ["nodedialer.go"], + srcs = [ + "nodedialer.go", + "nodedialer_drpc.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer", visibility = ["//visibility:public"], deps = [ @@ -20,6 +23,7 @@ go_library( "//pkg/util/stop", "//pkg/util/tracing", "@com_github_cockroachdb_errors//:errors", + "@io_storj_drpc//drpcpool", "@org_golang_google_grpc//:grpc", ], ) diff --git a/pkg/rpc/nodedialer/nodedialer.go b/pkg/rpc/nodedialer/nodedialer.go index 4bebfd4f5dd4..3faff95ea406 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "google.golang.org/grpc" + "storj.io/drpc/drpcpool" ) // An AddressResolver translates NodeIDs into addresses. @@ -100,7 +101,7 @@ func (n *Dialer) Dial( err = errors.Wrapf(err, "failed to resolve n%d", nodeID) return nil, err } - conn, _, err := n.dial(ctx, nodeID, addr, locality, true, class) + conn, _, _, _, err := n.dial(ctx, nodeID, addr, locality, true, class) return conn, err } @@ -117,7 +118,7 @@ func (n *Dialer) DialNoBreaker( if err != nil { return nil, err } - conn, _, err := n.dial(ctx, nodeID, addr, locality, false, class) + conn, _, _, _, err := n.dial(ctx, nodeID, addr, locality, false, class) return conn, err } @@ -147,14 +148,31 @@ func (n *Dialer) DialInternalClient( return nil, errors.Wrap(err, "resolver error") } log.VEventf(ctx, 2, "sending request to %s", addr) - conn, pool, err := n.dial(ctx, nodeID, addr, locality, true, class) + conn, pool, dconn, drpcBatchStreamPool, err := n.dial(ctx, nodeID, addr, locality, true, class) if err != nil { return nil, err } + client := newBaseInternalClient(conn) - if shouldUseBatchStreamPoolClient(ctx, n.rpcContext.Settings) { + useStreamPoolClient := shouldUseBatchStreamPoolClient(ctx, n.rpcContext.Settings) + if useStreamPoolClient { client = newBatchStreamPoolClient(pool) } + + if rpc.ExperimentalDRPCEnabled.Get(&n.rpcContext.Settings.SV) { + // TODO(server): gRPC version of batch stream pool implements + // rpc.RestrictedInternalClient and is allocation-optimized, + // whereas here we allocate a new throw-away + // unaryDRPCBatchServiceToInternalAdapter. + client = &unaryDRPCBatchServiceToInternalAdapter{ + useStreamPoolClient: useStreamPoolClient, + RestrictedInternalClient: client, // for RangeFeed only + drpcClient: kvpb.NewDRPCBatchClient(dconn), + drpcStreamPool: drpcBatchStreamPool, + } + return client, nil + } + client = maybeWrapInTracingClient(ctx, client) return client, nil } @@ -169,28 +187,29 @@ func (n *Dialer) dial( locality roachpb.Locality, checkBreaker bool, class rpc.ConnectionClass, -) (_ *grpc.ClientConn, _ *rpc.BatchStreamPool, err error) { +) (*grpc.ClientConn, *rpc.BatchStreamPool, drpcpool.Conn, *rpc.DRPCBatchStreamPool, error) { const ctxWrapMsg = "dial" // Don't trip the breaker if we're already canceled. if ctxErr := ctx.Err(); ctxErr != nil { - return nil, nil, errors.Wrap(ctxErr, ctxWrapMsg) + return nil, nil, nil, nil, errors.Wrap(ctxErr, ctxWrapMsg) } rpcConn := n.rpcContext.GRPCDialNode(addr.String(), nodeID, locality, class) - connect := rpcConn.Connect + connect := rpcConn.ConnectEx if !checkBreaker { connect = rpcConn.ConnectNoBreaker } - conn, err := connect(ctx) + conn, dconn, err := connect(ctx) if err != nil { // If we were canceled during the dial, don't trip the breaker. if ctxErr := ctx.Err(); ctxErr != nil { - return nil, nil, errors.Wrap(ctxErr, ctxWrapMsg) + return nil, nil, nil, nil, errors.Wrap(ctxErr, ctxWrapMsg) } err = errors.Wrapf(err, "failed to connect to n%d at %v", nodeID, addr) - return nil, nil, err + return nil, nil, nil, nil, err } pool := rpcConn.BatchStreamPool() - return conn, pool, nil + drpcStreamPool := rpcConn.DRPCBatchStreamPool() + return conn, pool, dconn, drpcStreamPool, nil } // ConnHealth returns nil if we have an open connection of the request diff --git a/pkg/rpc/nodedialer/nodedialer_drpc.go b/pkg/rpc/nodedialer/nodedialer_drpc.go new file mode 100644 index 000000000000..d4ff92fdd1df --- /dev/null +++ b/pkg/rpc/nodedialer/nodedialer_drpc.go @@ -0,0 +1,35 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package nodedialer + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/errors" + "google.golang.org/grpc" +) + +type unaryDRPCBatchServiceToInternalAdapter struct { + useStreamPoolClient bool + rpc.RestrictedInternalClient + drpcClient kvpb.DRPCBatchClient + drpcStreamPool *rpc.DRPCBatchStreamPool +} + +func (a *unaryDRPCBatchServiceToInternalAdapter) Batch( + ctx context.Context, in *kvpb.BatchRequest, opts ...grpc.CallOption, +) (*kvpb.BatchResponse, error) { + if len(opts) > 0 { + return nil, errors.New("CallOptions unsupported") + } + if a.useStreamPoolClient && a.drpcStreamPool != nil { + return a.drpcStreamPool.Send(ctx, in) + } + + return a.drpcClient.Batch(ctx, in) +} diff --git a/pkg/rpc/peer.go b/pkg/rpc/peer.go index 3cc0bb599168..05ca9e972be4 100644 --- a/pkg/rpc/peer.go +++ b/pkg/rpc/peer.go @@ -27,6 +27,7 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/status" + "storj.io/drpc/drpcpool" ) type peerStatus int @@ -125,6 +126,7 @@ type peer struct { heartbeatInterval time.Duration heartbeatTimeout time.Duration dial func(ctx context.Context, target string, class ConnectionClass) (*grpc.ClientConn, error) + dialDRPC func(ctx context.Context, target string) (drpcpool.Conn, error) // b maintains connection health. This breaker's async probe is always // active - it is the heartbeat loop and manages `mu.c.` (including // recreating it after the connection fails and has to be redialed). @@ -245,6 +247,7 @@ func (rpcCtx *Context) newPeer(k peerKey, locality roachpb.Locality) *peer { additionalDialOpts = append(additionalDialOpts, rpcCtx.testingDialOpts...) return rpcCtx.grpcDialRaw(ctx, target, class, additionalDialOpts...) }, + dialDRPC: dialDRPC(rpcCtx), heartbeatInterval: rpcCtx.RPCHeartbeatInterval, heartbeatTimeout: rpcCtx.RPCHeartbeatTimeout, } @@ -381,6 +384,13 @@ func (p *peer) runOnce(ctx context.Context, report func(error)) error { defer func() { _ = cc.Close() // nolint:grpcconnclose }() + dc, err := p.dialDRPC(ctx, p.k.TargetAddr) + if err != nil { + return err + } + defer func() { + _ = dc.Close() + }() // Set up notifications on a channel when gRPC tears down, so that we // can trigger another instant heartbeat for expedited circuit breaker @@ -399,7 +409,7 @@ func (p *peer) runOnce(ctx context.Context, report func(error)) error { return err } - p.onInitialHeartbeatSucceeded(ctx, p.opts.Clock.Now(), cc, report) + p.onInitialHeartbeatSucceeded(ctx, p.opts.Clock.Now(), cc, dc, report) return p.runHeartbeatUntilFailure(ctx, connFailedCh) } @@ -563,7 +573,7 @@ func logOnHealthy(ctx context.Context, disconnected, now time.Time) { } func (p *peer) onInitialHeartbeatSucceeded( - ctx context.Context, now time.Time, cc *grpc.ClientConn, report func(err error), + ctx context.Context, now time.Time, cc *grpc.ClientConn, dc drpcpool.Conn, report func(err error), ) { // First heartbeat succeeded. By convention we update the breaker // before updating the peer. The other way is fine too, just the @@ -586,10 +596,11 @@ func (p *peer) onInitialHeartbeatSucceeded( // ahead of signaling the connFuture, so that the stream pool is ready for use // by the time the connFuture is resolved. p.mu.c.batchStreamPool.Bind(ctx, cc) + p.mu.c.drpcBatchStreamPool.Bind(ctx, dc) // Close the channel last which is helpful for unit tests that // first waitOrDefault for a healthy conn to then check metrics. - p.mu.c.connFuture.Resolve(cc, nil /* err */) + p.mu.c.connFuture.Resolve(cc, dc, nil /* err */) logOnHealthy(ctx, p.mu.disconnected, now) } @@ -706,7 +717,7 @@ func (p *peer) onHeartbeatFailed( // someone might be waiting on it in ConnectNoBreaker who is not paying // attention to the circuit breaker. err = &netutil.InitialHeartbeatFailedError{WrappedErr: err} - ls.c.connFuture.Resolve(nil /* cc */, err) + ls.c.connFuture.Resolve(nil /* cc */, nil /* dc */, err) } // Close down the stream pool that was bound to this connection. @@ -746,7 +757,7 @@ func (p *peer) onQuiesce(report func(error)) { // NB: it's important that connFuture is resolved, or a caller sitting on // `c.ConnectNoBreaker` would never be unblocked; after all, the probe won't // start again in the future. - p.snap().c.connFuture.Resolve(nil, errQuiescing) + p.snap().c.connFuture.Resolve(nil, nil, errQuiescing) } func (p PeerSnap) deletable(now time.Time) bool { diff --git a/pkg/rpc/stream_pool.go b/pkg/rpc/stream_pool.go index 20773d714f0a..b12ea472881b 100644 --- a/pkg/rpc/stream_pool.go +++ b/pkg/rpc/stream_pool.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" "google.golang.org/grpc" + "storj.io/drpc" ) // streamClient is a type constraint that is satisfied by a bidirectional gRPC @@ -331,3 +332,12 @@ type BatchStreamClient = streamClient[*kvpb.BatchRequest, *kvpb.BatchResponse] func newBatchStream(ctx context.Context, cc *grpc.ClientConn) (BatchStreamClient, error) { return kvpb.NewInternalClient(cc).BatchStream(ctx) } + +type DRPCBatchStreamPool = streamPool[*kvpb.BatchRequest, *kvpb.BatchResponse, drpc.Conn] + +type DRPCBatchStreamClient = streamClient[*kvpb.BatchRequest, *kvpb.BatchResponse] + +// newDRPCBatchStream constructs a BatchStreamClient from a drpc.Conn. +func newDRPCBatchStream(ctx context.Context, dc drpc.Conn) (DRPCBatchStreamClient, error) { + return kvpb.NewDRPCBatchClient(dc).BatchStream(ctx) +} From 3d8f276362711f1431127a269ae39b9186f96823 Mon Sep 17 00:00:00 2001 From: Chandra Thumuluru Date: Fri, 10 Jan 2025 10:10:04 +0000 Subject: [PATCH 055/126] server: test hosting drpc server that serves BatchRequests --- pkg/rpc/drpc.go | 8 ---- pkg/server/drpc_test.go | 81 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 81 insertions(+), 8 deletions(-) create mode 100644 pkg/server/drpc_test.go diff --git a/pkg/rpc/drpc.go b/pkg/rpc/drpc.go index 4af695162801..f5cd982ea506 100644 --- a/pkg/rpc/drpc.go +++ b/pkg/rpc/drpc.go @@ -88,14 +88,6 @@ func dialDRPC(rpcCtx *Context) func(ctx context.Context, target string) (drpcpoo if err != nil { return nil, err } - // TODO(server): at least with testing certs, we get: manager closed: - // tls: either ServerName or InsecureSkipVerify must be specified in - // the tls.Config from drpcmanager.(*Manager).manageReader:234 - // - // This is possibly avoided in gRPC by setting ServerName in - // (*tlsCreds).ClientHandshake. - tlsConfig = tlsConfig.Clone() - tlsConfig.InsecureSkipVerify = true // HACK tlsConn := tls.Client(netConn, tlsConfig) conn = drpcconn.NewWithOptions(tlsConn, opts) } diff --git a/pkg/server/drpc_test.go b/pkg/server/drpc_test.go new file mode 100644 index 000000000000..7423e0f59fe5 --- /dev/null +++ b/pkg/server/drpc_test.go @@ -0,0 +1,81 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package server_test + +import ( + "context" + "crypto/tls" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" + "storj.io/drpc/drpcconn" + "storj.io/drpc/drpcmigrate" +) + +// TestDRPCBatchServer verifies that CRDB nodes can host a drpc server that +// serves BatchRequest. It doesn't verify that nodes use drpc to communiate with +// each other. +func TestDRPCBatchServer(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + const numNodes = 1 + + testutils.RunTrueAndFalse(t, "insecure", func(t *testing.T, insecure bool) { + args := base.TestClusterArgs{} + args.ServerArgs.Insecure = insecure + args.ReplicationMode = base.ReplicationManual + args.ServerArgs.Settings = cluster.MakeClusterSettings() + rpc.ExperimentalDRPCEnabled.Override(ctx, &args.ServerArgs.Settings.SV, true) + c := testcluster.StartTestCluster(t, numNodes, args) + defer c.Stopper().Stop(ctx) + + require.Equal(t, insecure, c.Server(0).RPCContext().Insecure) + + rpcAddr := c.Server(0).RPCAddr() + + // Dial the drpc server with the drpc connection header. + rawconn, err := drpcmigrate.DialWithHeader(ctx, "tcp", rpcAddr, drpcmigrate.DRPCHeader) + require.NoError(t, err) + + var conn *drpcconn.Conn + if !insecure { + cm, err := c.Server(0).RPCContext().GetCertificateManager() + require.NoError(t, err) + tlsCfg, err := cm.GetNodeClientTLSConfig() + require.NoError(t, err) + tlsCfg = tlsCfg.Clone() + tlsCfg.ServerName = "*.local" + tlsConn := tls.Client(rawconn, tlsCfg) + conn = drpcconn.New(tlsConn) + } else { + conn = drpcconn.New(rawconn) + } + defer func() { require.NoError(t, conn.Close()) }() + + desc := c.LookupRangeOrFatal(t, c.ScratchRange(t)) + + client := kvpb.NewDRPCBatchClient(conn) + ba := &kvpb.BatchRequest{} + ba.RangeID = desc.RangeID + var ok bool + ba.Replica, ok = desc.GetReplicaDescriptor(1) + require.True(t, ok) + req := &kvpb.LeaseInfoRequest{} + req.Key = desc.StartKey.AsRawKey() + ba.Add(req) + _, err = client.Batch(ctx, ba) + require.NoError(t, err) + }) +} From b8115b4b92984dd257423993a195ee47103b4cb9 Mon Sep 17 00:00:00 2001 From: Chandra Thumuluru Date: Mon, 13 Jan 2025 04:59:44 +0000 Subject: [PATCH 056/126] rpc,server: add mock DRPC server and muxer implementation Use mock drpc server and muxer implementation when DRPC is disabled. Epic: None Release note: None --- pkg/rpc/context.go | 8 --- pkg/rpc/drpc.go | 106 +++++++++++++++++++++++++++---------- pkg/server/start_listen.go | 1 + 3 files changed, 79 insertions(+), 36 deletions(-) diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index a2bc7cf52af7..8810686384e4 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -49,8 +49,6 @@ import ( "google.golang.org/grpc/encoding" "google.golang.org/grpc/metadata" "google.golang.org/grpc/stats" - "storj.io/drpc/drpcmux" - "storj.io/drpc/drpcserver" ) // NewServer sets up an RPC server. Depending on the ServerOptions, the Server @@ -79,12 +77,6 @@ type ClientInterceptorInfo struct { StreamInterceptors []grpc.StreamClientInterceptor } -type DRPCServer struct { - Srv *drpcserver.Server - Mux *drpcmux.Mux - TLSCfg *tls.Config -} - // NewServerEx is like NewServer, but also returns the interceptors that have // been registered with gRPC for the server. These interceptors can be used // manually when bypassing gRPC to call into the server (like the diff --git a/pkg/rpc/drpc.go b/pkg/rpc/drpc.go index f5cd982ea506..4b8e342d2fbc 100644 --- a/pkg/rpc/drpc.go +++ b/pkg/rpc/drpc.go @@ -9,8 +9,11 @@ import ( "context" "crypto/tls" "math" + "net" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "storj.io/drpc" "storj.io/drpc/drpcconn" "storj.io/drpc/drpcmanager" "storj.io/drpc/drpcmigrate" @@ -21,34 +24,63 @@ import ( "storj.io/drpc/drpcwire" ) -func newDRPCServer(ctx context.Context, rpcCtx *Context) (*DRPCServer, error) { - dmux := drpcmux.New() - // NB: any server middleware (server interceptors in gRPC parlance) would go - // here: - // dmux = whateverMiddleware1(dmux) - // dmux = whateverMiddleware2(dmux) - // ... - // - // Each middleware must implement the Handler interface: - // - // HandleRPC(stream Stream, rpc string) error - // - // where Stream - // See here for an example: - // https://github.com/bryk-io/pkg/blob/4da5fbfef47770be376e4022eab5c6c324984bf7/net/drpc/server.go#L91-L101 - - dsrv := drpcserver.NewWithOptions(dmux, drpcserver.Options{ - Log: func(err error) { - log.Warningf(context.Background(), "drpc server error %v", err) - }, - // The reader's max buffer size defaults to 4mb, and if it is exceeded (such - // as happens with AddSSTable) the RPCs fail. - Manager: drpcmanager.Options{Reader: drpcwire.ReaderOptions{MaximumBufferSize: math.MaxInt}}, - }) - - tlsCfg, err := rpcCtx.GetServerTLSConfig() - if err != nil { - return nil, err +// ErrDRPCDisabled is returned from hosts that in principle could but do not +// have the DRPC server enabled. +var ErrDRPCDisabled = errors.New("DRPC is not enabled") + +type drpcServerI interface { + Serve(ctx context.Context, lis net.Listener) error +} + +type drpcMuxI interface { + Register(srv interface{}, desc drpc.Description) error +} + +type DRPCServer struct { + Srv drpcServerI + Mux drpcMuxI + TLSCfg *tls.Config +} + +var _ drpcServerI = (*drpcserver.Server)(nil) +var _ drpcServerI = (*drpcOffServer)(nil) + +func newDRPCServer(_ context.Context, rpcCtx *Context) (*DRPCServer, error) { + var dmux drpcMuxI = &drpcOffServer{} + var dsrv drpcServerI = &drpcOffServer{} + var tlsCfg *tls.Config + + if ExperimentalDRPCEnabled.Get(&rpcCtx.Settings.SV) { + mux := drpcmux.New() + dsrv = drpcserver.NewWithOptions(mux, drpcserver.Options{ + Log: func(err error) { + log.Warningf(context.Background(), "drpc server error %v", err) + }, + // The reader's max buffer size defaults to 4mb, and if it is exceeded (such + // as happens with AddSSTable) the RPCs fail. + Manager: drpcmanager.Options{Reader: drpcwire.ReaderOptions{MaximumBufferSize: math.MaxInt}}, + }) + dmux = mux + + var err error + tlsCfg, err = rpcCtx.GetServerTLSConfig() + if err != nil { + return nil, err + } + + // NB: any server middleware (server interceptors in gRPC parlance) would go + // here: + // dmux = whateverMiddleware1(dmux) + // dmux = whateverMiddleware2(dmux) + // ... + // + // Each middleware must implement the Handler interface: + // + // HandleRPC(stream Stream, rpc string) error + // + // where Stream + // See here for an example: + // https://github.com/bryk-io/pkg/blob/4da5fbfef47770be376e4022eab5c6c324984bf7/net/drpc/server.go#L91-L101 } return &DRPCServer{ @@ -115,3 +147,21 @@ func (c *closeEntirePoolConn) Close() error { _ = c.Conn.Close() return c.pool.Close() } + +// drpcOffServer is used for drpcServerI and drpcMuxI if the DRPC server is +// disabled. It immediately closes accepted connections and returns +// ErrDRPCDisabled. +type drpcOffServer struct{} + +func (srv *drpcOffServer) Serve(_ context.Context, lis net.Listener) error { + conn, err := lis.Accept() + if err != nil { + return err + } + _ = conn.Close() + return ErrDRPCDisabled +} + +func (srv *drpcOffServer) Register(interface{}, drpc.Description) error { + return nil +} diff --git a/pkg/server/start_listen.go b/pkg/server/start_listen.go index 59471cefde2b..64b9eec20bd1 100644 --- a/pkg/server/start_listen.go +++ b/pkg/server/start_listen.go @@ -167,6 +167,7 @@ func startListenRPCAndSQL( drpcCancel() // TODO(bdarnell): Do we need to also close the other listeners? netutil.FatalIfUnexpected(grpcL.Close()) + netutil.FatalIfUnexpected(drpcL.Close()) netutil.FatalIfUnexpected(rpcLoopbackL.Close()) netutil.FatalIfUnexpected(sqlLoopbackL.Close()) netutil.FatalIfUnexpected(ln.Close()) From 0107f13c84d109590b42e51bb00530cccff7e397 Mon Sep 17 00:00:00 2001 From: Annie Pompa Date: Fri, 13 Dec 2024 10:31:18 -0500 Subject: [PATCH 057/126] sql: extend drop region for system db This patch ensures that references to a dropping region get cleaned up for the system database. Epic: none Fixes: #137095 Release note: None --- .../multiregion_system_table_test.go | 78 +++++++++++++++++++ pkg/sql/alter_database.go | 35 +++++++++ pkg/sql/type_change.go | 67 +++++++++++++++- 3 files changed, 176 insertions(+), 4 deletions(-) diff --git a/pkg/ccl/multiregionccl/multiregion_system_table_test.go b/pkg/ccl/multiregionccl/multiregion_system_table_test.go index 64e3f1230482..eb9817a714b3 100644 --- a/pkg/ccl/multiregionccl/multiregion_system_table_test.go +++ b/pkg/ccl/multiregionccl/multiregion_system_table_test.go @@ -554,3 +554,81 @@ func TestMrSystemDatabaseUpgrade(t *testing.T) { {"ALTER PARTITION \"us-east3\" OF INDEX system.public.lease@primary CONFIGURE ZONE USING\n\tnum_voters = 3,\n\tvoter_constraints = '[+region=us-east3]',\n\tlease_preferences = '[[+region=us-east3]]'"}, }) } + +func TestMrSystemDatabaseDropRegion(t *testing.T) { + defer leaktest.AfterTest(t)() + + defer log.Scope(t).Close(t) + + ctx := context.Background() + + // Enable settings required for configuring a tenant's system database as multi-region. + makeSettings := func() *cluster.Settings { + cs := cluster.MakeTestingClusterSettingsWithVersions(clusterversion.Latest.Version(), + clusterversion.MinSupported.Version(), + false) + instancestorage.ReclaimLoopInterval.Override(ctx, &cs.SV, 150*time.Millisecond) + return cs + } + + cluster, _, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster(t, 3, + base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + ClusterVersionOverride: clusterversion.MinSupported.Version(), + }, + }, + multiregionccltestutils.WithSettings(makeSettings())) + defer cleanup() + id, err := roachpb.MakeTenantID(11) + require.NoError(t, err) + + // Disable license enforcement for this test. + for _, s := range cluster.Servers { + s.ExecutorConfig().(sql.ExecutorConfig).LicenseEnforcer.Disable(ctx) + } + + tenantArgs := base.TestTenantArgs{ + Settings: makeSettings(), + TestingKnobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + ClusterVersionOverride: clusterversion.MinSupported.Version(), + }, + }, + TenantID: id, + Locality: cluster.Servers[0].Locality(), + } + appLayer, tenantSQL := serverutils.StartTenant(t, cluster.Servers[0], tenantArgs) + appLayer.ExecutorConfig().(sql.ExecutorConfig).LicenseEnforcer.Disable(ctx) + + tDB := sqlutils.MakeSQLRunner(tenantSQL) + + tDB.Exec(t, `ALTER DATABASE system SET PRIMARY REGION "us-east1"`) + tDB.Exec(t, `ALTER DATABASE system ADD REGION "us-east2"`) + tDB.Exec(t, `ALTER DATABASE system ADD REGION "us-east3"`) + tDB.Exec(t, `ALTER DATABASE defaultdb SET PRIMARY REGION "us-east1"`) + tDB.Exec(t, `ALTER DATABASE defaultdb ADD REGION "us-east2"`) + tDB.Exec(t, `ALTER DATABASE defaultdb ADD REGION "us-east3"`) + + tDB.CheckQueryResults(t, "SELECT create_statement FROM [SHOW CREATE DATABASE system]", [][]string{ + {"CREATE DATABASE system PRIMARY REGION \"us-east1\" REGIONS = \"us-east1\", \"us-east2\", \"us-east3\" SURVIVE REGION FAILURE"}, + }) + + tDB.ExpectErr(t, "region is still in use", `ALTER DATABASE system DROP REGION "us-east3"`) + tDB.Exec(t, `ALTER DATABASE defaultdb DROP REGION "us-east3"`) + tDB.Exec(t, `ALTER DATABASE system DROP REGION "us-east3"`) + + tDB.CheckQueryResults(t, `SELECT count(*) FROM system.sql_instances WHERE crdb_region != 'us-east1'::system.public.crdb_internal_region AND crdb_region != 'us-east2'::system.public.crdb_internal_region`, [][]string{ + {"0"}, + }) + tDB.CheckQueryResults(t, `SELECT count(*) FROM system.sqlliveness WHERE crdb_region != 'us-east1'::system.public.crdb_internal_region AND crdb_region != 'us-east2'::system.public.crdb_internal_region`, [][]string{ + {"0"}, + }) + tDB.CheckQueryResults(t, `SELECT count(*) FROM system.region_liveness WHERE crdb_region != 'us-east1'::system.public.crdb_internal_region AND crdb_region != 'us-east2'::system.public.crdb_internal_region`, [][]string{ + {"0"}, + }) + tDB.CheckQueryResults(t, `SELECT count(*) FROM system.lease WHERE crdb_region != 'us-east1'::system.public.crdb_internal_region AND crdb_region != 'us-east2'::system.public.crdb_internal_region`, [][]string{ + {"0"}, + }) +} diff --git a/pkg/sql/alter_database.go b/pkg/sql/alter_database.go index 9960cac8468a..7d014e1c8163 100644 --- a/pkg/sql/alter_database.go +++ b/pkg/sql/alter_database.go @@ -29,7 +29,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/regionliveness" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlclustersettings" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -483,6 +485,39 @@ func (p *planner) AlterDatabaseDropRegion( if err := p.checkCanDropSystemDatabaseRegion(ctx, n.Region); err != nil { return nil, err } + tablesToClean := []string{"sqlliveness", "lease", "sql_instances"} + for _, t := range tablesToClean { + livenessQuery := fmt.Sprintf( + `SELECT count(*) > 0 FROM system.%s WHERE crdb_region = '%s' AND + crdb_internal.sql_liveness_is_alive(session_id)`, t, n.Region) + row, err := p.QueryRowEx(ctx, "check-session-liveness-for-region", + sessiondata.NodeUserSessionDataOverride, livenessQuery) + if err != nil { + return nil, err + } + // Block dropping n.Region if any associated session is active. + if tree.MustBeDBool(row[0]) { + return nil, errors.WithHintf( + pgerror.Newf( + pgcode.InvalidDatabaseDefinition, + "cannot drop region %q", + n.Region, + ), + "You must not have any active sessions that are in this region. "+ + "Ensure that there no nodes that still belong to region %q", n.Region, + ) + } + } + // For the region_liveness table, we can just safely remove the reference + // (if any) of the dropping region from the table. + if _, err := p.ExecEx(ctx, "remove-region-liveness-ref", + sessiondata.NodeUserSessionDataOverride, `DELETE FROM system.region_liveness + WHERE crdb_region = $1`, n.Region); err != nil { + return nil, err + } + if err := regionliveness.CleanupSystemTableForRegion(ctx, p.execCfg.Codec, n.Region.String(), p.txn); err != nil { + return nil, err + } } // Ensure survivability goal and number of regions after the drop jive. diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index 0cf74db302ba..435a83119906 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -312,7 +312,7 @@ func (t *typeSchemaChanger) exec(ctx context.Context) error { // exposing things in the right order in OnFailOrCancel. This is because // OnFailOrCancel doesn't expose any new state in the type descriptor // (it just cleans up non-public states). - var multiRegionPreDropIsNecessary bool + var isDroppingMultiRegionEnumMember bool withDatabaseRegionChangeFinalizer := func( ctx context.Context, txn descs.Txn, f func(finalizer *databaseRegionChangeFinalizer) error, @@ -367,7 +367,7 @@ func (t *typeSchemaChanger) exec(ctx context.Context) error { for _, member := range typeDesc.EnumMembers { if t.isTransitioningInCurrentJob(&member) && enumMemberIsRemoving(&member) { if typeDesc.Kind == descpb.TypeDescriptor_MULTIREGION_ENUM { - multiRegionPreDropIsNecessary = true + isDroppingMultiRegionEnumMember = true } toDrop = append(toDrop, member) } @@ -385,7 +385,7 @@ func (t *typeSchemaChanger) exec(ctx context.Context) error { // transaction to be a writing transaction; it would have a heck of // a lot of data to refresh. We instead defer the repartitioning until // after this checking confirms the safety of the change. - if multiRegionPreDropIsNecessary { + if isDroppingMultiRegionEnumMember { repartitioned, err := prepareRepartitionedRegionalByRowTables(ctx, txn) if err != nil { return err @@ -403,10 +403,51 @@ func (t *typeSchemaChanger) exec(ctx context.Context) error { } return nil } + + var idsToRemove []int + populateIDsToRemove := func(holder context.Context, txn descs.Txn) error { + typeDesc, err := txn.Descriptors().MutableByID(txn.KV()).Type(ctx, t.typeID) + if err != nil { + return err + } + for _, member := range typeDesc.EnumMembers { + if !t.isTransitioningInCurrentJob(&member) || + !enumMemberIsRemoving(&member) || + typeDesc.Kind != descpb.TypeDescriptor_MULTIREGION_ENUM { + continue + } + rows, err := txn.QueryBufferedEx(ctx, "select-invalid-instances", txn.KV(), + sessiondata.NodeUserSessionDataOverride, `SELECT id FROM system.sql_instances + WHERE crdb_region = $1`, member.PhysicalRepresentation) + if err != nil { + return err + } + for _, row := range rows { + idsToRemove = append(idsToRemove, int(tree.MustBeDInt(row[0]))) + } + } + return nil + } + + removeReferences := func(ctx context.Context, txn descs.Txn) error { + for _, id := range idsToRemove { + deleteQuery := fmt.Sprintf( + `DELETE FROM system.sql_instances WHERE id = %d`, id) + if _, err := txn.ExecEx(ctx, "delete-dropped-region-ref", txn.KV(), + sessiondata.NodeUserSessionDataOverride, deleteQuery); err != nil { + return err + } + + } + return nil + } if err := t.execCfg.InternalDB.DescsTxn(ctx, validateDrops); err != nil { return err } - if multiRegionPreDropIsNecessary { + if isDroppingMultiRegionEnumMember { + if err := t.execCfg.InternalDB.DescsTxn(ctx, populateIDsToRemove); err != nil { + return err + } if err := t.execCfg.InternalDB.DescsTxn(ctx, repartitionRegionalByRowTables); err != nil { return err } @@ -500,6 +541,12 @@ func (t *typeSchemaChanger) exec(ctx context.Context) error { if err := refreshTypeDescriptorLeases(ctx, leaseMgr, t.execCfg.DB, typeDesc); err != nil { return err } + + if isDroppingMultiRegionEnumMember && len(idsToRemove) != 0 { + if err := t.execCfg.InternalDB.DescsTxn(ctx, removeReferences); err != nil { + return err + } + } } // If the type is being dropped, remove the descriptor here only @@ -1009,6 +1056,18 @@ func (t *typeSchemaChanger) canRemoveEnumValueFromTable( // Check if the above query returned a result. If it did, then the // enum value is being used by some place. if len(rows) > 0 { + // If our enum member is being removed, we can skip this check + // because we need to wait until the region is removed from our + // multiregion enum before we can drop the reference entirely. + // We will perform said cleanup later on during the type schema + // change. We have to do this because + // instancestorage.RunInstanceIDReclaimLoop will add prewarmed + // entries in the instances table for each public region. + if member.Direction == descpb.TypeDescriptor_EnumMember_REMOVE { + if desc.GetID() == keys.SQLInstancesTableID { + return nil + } + } return pgerror.Newf(pgcode.DependentObjectsStillExist, "could not remove enum value %q as it is being used by %q in row: %s", member.LogicalRepresentation, desc.GetName(), labeledRowValues(desc.AccessibleColumns(), rows)) From 5a3e042b3335def4fb8f0dba927205518524f0b5 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Mon, 13 Jan 2025 11:15:10 -0500 Subject: [PATCH 058/126] roachtest: increase timeout for validate-system-schema-after-version-upgrade Release note: None --- pkg/cmd/roachtest/tests/acceptance.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/acceptance.go b/pkg/cmd/roachtest/tests/acceptance.go index 529aeaac3ac2..56eb2c631378 100644 --- a/pkg/cmd/roachtest/tests/acceptance.go +++ b/pkg/cmd/roachtest/tests/acceptance.go @@ -98,7 +98,7 @@ func registerAcceptance(r registry.Registry) { { name: "validate-system-schema-after-version-upgrade", fn: runValidateSystemSchemaAfterVersionUpgrade, - timeout: 30 * time.Minute, + timeout: 60 * time.Minute, defaultLeases: true, randomized: true, numNodes: 1, From a577d094fb1527678d8a7a84506cecba097e2b38 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 6 Jan 2025 16:27:45 -0800 Subject: [PATCH 059/126] props: add fast path for ReduceCols This commit adds a fast path to `FuncDepSet.ReduceCols`. If the FD set has a strict key and the key is a subset of the given columns, the columns are immediately reduced just to the key columns. From that point the columns may be further reduced if the set has been updated since the key was originally calculated. Note that changes had to be made to `MakeProduct` and `MakeApply` to ensure that the original key was invalidated before new dependencies were added to the set. Epic: None Release note: None --- pkg/sql/opt/norm/testdata/rules/groupby | 14 +++++------ pkg/sql/opt/props/func_dep.go | 32 +++++++++++++++++++------ 2 files changed, 32 insertions(+), 14 deletions(-) diff --git a/pkg/sql/opt/norm/testdata/rules/groupby b/pkg/sql/opt/norm/testdata/rules/groupby index 65e67c96e9df..d9c6851d26a7 100644 --- a/pkg/sql/opt/norm/testdata/rules/groupby +++ b/pkg/sql/opt/norm/testdata/rules/groupby @@ -1534,14 +1534,14 @@ SELECT min(f) FROM a GROUP BY i, s, k project ├── columns: min:8 └── group-by (hash) - ├── columns: i:2!null s:4!null min:8 - ├── grouping columns: i:2!null s:4!null - ├── key: (2,4) - ├── fd: (2,4)-->(8) + ├── columns: k:1!null min:8 + ├── grouping columns: k:1!null + ├── key: (1) + ├── fd: (1)-->(8) ├── scan a - │ ├── columns: i:2!null f:3 s:4!null - │ ├── key: (2,4) - │ └── fd: (2,4)-->(3), (2,3)~~>(4) + │ ├── columns: k:1!null f:3 + │ ├── key: (1) + │ └── fd: (1)-->(3) └── aggregations └── min [as=min:8, outer=(3)] └── f:3 diff --git a/pkg/sql/opt/props/func_dep.go b/pkg/sql/opt/props/func_dep.go index b99f51c91056..a52b5e5467a1 100644 --- a/pkg/sql/opt/props/func_dep.go +++ b/pkg/sql/opt/props/func_dep.go @@ -641,6 +641,11 @@ func (f *FuncDepSet) ConstantCols() opt.ColSet { // so, then the column is redundant. This algorithm has decent running time, but // will not necessarily find the candidate key with the fewest columns. func (f *FuncDepSet) ReduceCols(cols opt.ColSet) opt.ColSet { + if f.hasKey == strictKey && f.key.SubsetOf(cols) { + // Fast path: take advantage of the reduction that was already performed for + // an existing key. + cols = f.key + } var removed opt.ColSet cols = cols.Copy() for i, ok := cols.Next(0); ok; i, ok = cols.Next(i + 1) { @@ -1146,6 +1151,13 @@ func (f *FuncDepSet) AddEquivFrom(fdset *FuncDepSet) { // case of constant columns). func (f *FuncDepSet) MakeProduct(inner *FuncDepSet) { f.equiv.AppendFromDisjoint(&inner.equiv) + + // Save the old key and clear it so that addDependency doesn't attempt to + // reduce it. We'll handle the key below, and key reduction is expensive. + // Note that the ColSet used for the key is immutable, so it is safe to keep + // the reference to the old key. + oldKey, oldHasKey := f.key, f.hasKey + f.clearKey() for i := range inner.deps { fd := &inner.deps[i] if fd.isConstant() { @@ -1155,15 +1167,15 @@ func (f *FuncDepSet) MakeProduct(inner *FuncDepSet) { } } - if f.hasKey != noKey && inner.hasKey != noKey { + if oldHasKey != noKey && inner.hasKey != noKey { // If both sides have a strict key, the union of keys is a strict key. // If one side has a lax key and the other has a lax or strict key, the // union is a lax key. typ := laxKey - if f.hasKey == strictKey && inner.hasKey == strictKey { + if oldHasKey == strictKey && inner.hasKey == strictKey { typ = strictKey } - f.setKey(f.key.Union(inner.key), typ) + f.setKey(oldKey.Union(inner.key), typ) } else { f.clearKey() } @@ -1199,16 +1211,22 @@ func (f *FuncDepSet) MakeApply(inner *FuncDepSet) { // NOTE: the ColSet of an equiv group is immutable. f.addEquivalency(inner.equiv.Group(i)) } + // Save the old key and clear it so that addDependency doesn't attempt to + // reduce it. We'll handle the key below, and key reduction is expensive. + // Note that the ColSet used for the key is immutable, so it is safe to keep + // the reference to the old key. + oldKey, oldHasKey := f.key, f.hasKey + f.clearKey() for i := range inner.deps { fd := &inner.deps[i] - if !fd.isConstant() && f.hasKey == strictKey { - f.addDependency(f.key.Union(fd.from), fd.to, fd.strict) + if !fd.isConstant() && oldHasKey == strictKey { + f.addDependency(oldKey.Union(fd.from), fd.to, fd.strict) } // TODO(radu): can we use a laxKey here? } - if f.hasKey == strictKey && inner.hasKey == strictKey { - f.setKey(f.key.Union(inner.key), strictKey) + if oldHasKey == strictKey && inner.hasKey == strictKey { + f.setKey(oldKey.Union(inner.key), strictKey) f.ensureKeyClosure(inner.ColSet()) } else { // TODO(radu): can we use a laxKey here? From 0791f3def86821b2a58653e499f52e27b3491a57 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Mon, 6 Jan 2025 16:46:47 -0800 Subject: [PATCH 060/126] opt: optimize table func deps for non-unique indexes When a table is referenced in a query, we build a func-dep set describing the relationships between the table columns. Among other things, we use the index key columns for this purpose. There is no benefit to considering non-unique indexes here because the key columns are always a superset of the primary key columns. Therefore, this commit skips non-unique indexes to avoid the extra work. This results in a performance improvement for queries against tables with many non-unique indexes and many columns. Epic: None Release note: None --- pkg/sql/opt/memo/logical_props_builder.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 67842bb65a55..018b98bb2b06 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -2013,6 +2013,12 @@ func MakeTableFuncDep(md *opt.Metadata, tabID opt.TableID) *props.FuncDepSet { var keyCols opt.ColSet index := tab.Index(i) + if !index.IsUnique() { + // A non-unique index won't add any additional information, since it + // relies on the PK columns to form a key. + continue + } + if index.IsInverted() { // Skip inverted indexes for now. continue From 4bede39c31de5671678370bc878b959112720079 Mon Sep 17 00:00:00 2001 From: Wenyi Hu Date: Mon, 13 Jan 2025 11:51:23 -0500 Subject: [PATCH 061/126] changefeedccl/kvfeed: pass consumer id correctly Previously, we introduced the concept of a consumer ID to prevent a single changefeed job from over-consuming the catch-up scan quota and blocking other consumers from making progress on the server side. However, the changefeed client-side code requires the consumer ID to be passed again in the rangefeed options during rangefeedFactory.Run. This was missing in the previous PR, causing the changefeed job ID to default to zero. This patch fixes the issue by ensuring the consumer ID is correctly passed in the rangefeed options. Related: #133789 Release note: None --- pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go index e2402b9eef1d..1f337b9e734c 100644 --- a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go +++ b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go @@ -97,6 +97,7 @@ func (p rangefeedFactory) Run(ctx context.Context, sink kvevent.Writer, cfg rang if cfg.RangeObserver != nil { rfOpts = append(rfOpts, kvcoord.WithRangeObserver(cfg.RangeObserver)) } + rfOpts = append(rfOpts, kvcoord.WithConsumerID(cfg.ConsumerID)) if len(cfg.Knobs.RangefeedOptions) != 0 { rfOpts = append(rfOpts, cfg.Knobs.RangefeedOptions...) } From 2de665c71b5d415df67fe4e99b8e5752677af238 Mon Sep 17 00:00:00 2001 From: rimadeodhar Date: Fri, 3 Jan 2025 10:15:22 -0800 Subject: [PATCH 062/126] roachtest: Deflake multitenant upgrade test The multitenant upgrade test enforces different test scenarios while upgrading tenants in a mixed version state. The test enforces the following cases: 1. Start storage cluster with binary version: x, cluster version: x 2. Create some tenants with binary version: x and ensure they can connect to the cluster and run a workload. 3. Using the mixed version test framework, upgrade the storage cluster with binary version: x+1, cluster version: x. In this mixed version state, create remaining tenants with binary version: x and run a workload. 4. Finalize the storage cluster. At this point, the storage cluster has binary version: x+1 and cluster version: x+1 5. Upgrade tenants with binary version: x+1 and confirm tenants can connect to the storage cluster and run a workload. In https://github.com/cockroachdb/cockroach/pull/131847, the test was rewritten using the new mixed version test framework. However, this change exposed this test to a scenario that can cause this test to fail at step 3 above. The MVT framework also runs the mixed version test (i.e. with the tenant at the older binary version) when the cluster is in the finalizing stage. This scenario is run with a prefixed probability. However, if we attempt to start the tenants with the previous version (i.e. the version the cluster is being upgraded from) when the cluster is being finalized, the tenants rightfully fail to connect which the test incorrectly interprets as a failure. As a result, we would see this test fail occassionally since the test was updated to use the new MVT. This PR modifies the test to ensure that in the finalizing state, we start the tenants with the right version. Epic: none Fixes: https://github.com/cockroachdb/cockroach/issues/136447 Release note: None --- pkg/cmd/roachtest/tests/multitenant_upgrade.go | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/pkg/cmd/roachtest/tests/multitenant_upgrade.go b/pkg/cmd/roachtest/tests/multitenant_upgrade.go index 0a002acd1274..291f319e6d81 100644 --- a/pkg/cmd/roachtest/tests/multitenant_upgrade.go +++ b/pkg/cmd/roachtest/tests/multitenant_upgrade.go @@ -220,8 +220,20 @@ func runMultitenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { for _, tenant := range tenants { if !tenant.running { - if err := startTenant(ctx, l, tenant, h.Context().FromVersion, true); err != nil { - return err + if h.IsFinalizing() { + // If the upgrading service is finalizing, we need to stage the tenants with the upgraded + // binary to allow the tenant to start successfully. + if err := startTenant(ctx, l, tenant, h.Context().ToVersion, true); err != nil { + return err + } + } else { + // For all other upgrade stages, we can stage the tenant with the previous binary version i.e. + // the version from which the system tenant is being upgraded. This tests the scenario that + // in a mixed version state, tenants on the previous version can continue to connect + // to the cluster. + if err := startTenant(ctx, l, tenant, h.Context().FromVersion, true); err != nil { + return err + } } } } From d4ea7302d21f602247620f05fd0ee81a3f560246 Mon Sep 17 00:00:00 2001 From: DarrylWong Date: Thu, 9 Jan 2025 12:47:41 -0500 Subject: [PATCH 063/126] roachtest: bump FetchDebugZip timeout We've seen this timeout when collecting artifacts for failures despite the collection making reasonable progress. This change bumps the timeout to 10 minutes. --- pkg/cmd/roachtest/cluster.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index c385054785f7..bcc6caea2aff 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1387,7 +1387,7 @@ func (c *clusterImpl) FetchDebugZip( nodes := selectedNodesOrDefault(opts, c.All()) // Don't hang forever if we can't fetch the debug zip. - return timeutil.RunWithTimeout(ctx, "debug zip", 5*time.Minute, func(ctx context.Context) error { + return timeutil.RunWithTimeout(ctx, "debug zip", 10*time.Minute, func(ctx context.Context) error { const zipName = "debug.zip" path := filepath.Join(c.t.ArtifactsDir(), dest) if err := os.MkdirAll(filepath.Dir(path), 0755); err != nil { From cc5e53e78b8ac00038865b36288d0cbaa16b720a Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Mon, 13 Jan 2025 09:36:34 -0500 Subject: [PATCH 064/126] sql/externalcatalog: deflake TestExtractIngestExternalCatalog Previoulsy, the test would always set sql.gc_job.wait_for_gc.interval to 250 ms, to speed up gc of a dropped table. Occasionally, the gc process would complete so quickly that this post drop query would return no results, flaking the test: `SELECT name FROM crdb_internal.tables WHERE state = 'DROP'` This patch introduce to branches this test can take: - with fast gc, to test that the gc job was properly created - without fast gc, to test that the dropped descriptors end up in the dropped state. Fixes #138639 Release note: none --- .../externalcatalog/external_catalog_test.go | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/pkg/sql/catalog/externalcatalog/external_catalog_test.go b/pkg/sql/catalog/externalcatalog/external_catalog_test.go index 6cc14f4a710c..a39aeb44021e 100644 --- a/pkg/sql/catalog/externalcatalog/external_catalog_test.go +++ b/pkg/sql/catalog/externalcatalog/external_catalog_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/redact" "github.com/stretchr/testify/require" ) @@ -48,7 +49,12 @@ func TestExtractIngestExternalCatalog(t *testing.T) { sysDB := sqlutils.MakeSQLRunner(srv.SystemLayer().SQLConn(t)) sysDB.Exec(t, "SET CLUSTER SETTING sql.virtual_cluster.feature_access.zone_configs.enabled = true;") sysDB.Exec(t, "SET CLUSTER SETTING sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled = true;") - sysDB.Exec(t, "SET CLUSTER SETTING sql.gc_job.wait_for_gc.interval = '250ms'") + + rng, _ := randutil.NewTestRand() + fastGC := rng.Float64() < 0.5 + if fastGC { + sysDB.Exec(t, "SET CLUSTER SETTING sql.gc_job.wait_for_gc.interval = '250ms'") + } sqlDB := sqlutils.MakeSQLRunner(conn) sqlDB.Exec(t, "CREATE DATABASE db1") @@ -113,12 +119,17 @@ func TestExtractIngestExternalCatalog(t *testing.T) { var res int sqlDB.QueryRow(t, "SELECT count(*) FROM [SHOW TABLES]").Scan(&res) require.Zero(t, res) - sqlDB.CheckQueryResults(t, "SELECT name FROM crdb_internal.tables WHERE state = 'DROP'", [][]string{{"tab1"}, {"tab2"}}) - sqlDB.CheckQueryResultsRetry( - t, - "SELECT status FROM [SHOW JOBS] WHERE description = 'test gc'", - [][]string{{"succeeded"}}, - ) + if fastGC { + // With fast gc, the dropped tables may disappear before + // crdb_internal.tables is read. + sqlDB.CheckQueryResultsRetry( + t, + "SELECT status FROM [SHOW JOBS] WHERE description = 'test gc'", + [][]string{{"succeeded"}}, + ) + } else { + sqlDB.CheckQueryResults(t, "SELECT name FROM crdb_internal.tables WHERE state = 'DROP'", [][]string{{"tab1"}, {"tab2"}}) + } }) t.Run("fk", func(t *testing.T) { From e8cc195e31fd6035014151fb2e0693ddb6fad33f Mon Sep 17 00:00:00 2001 From: Justin Beaver Date: Mon, 13 Jan 2025 18:40:59 +0000 Subject: [PATCH 065/126] orchestration: released CockroachDB version 24.3.3. Next version: 24.3.4 Release note: None Epic: None Release justification: non-production (release infra) change. --- cloud/kubernetes/bring-your-own-certs/client.yaml | 2 +- .../bring-your-own-certs/cockroachdb-statefulset.yaml | 2 +- cloud/kubernetes/client-secure.yaml | 2 +- cloud/kubernetes/cluster-init-secure.yaml | 2 +- cloud/kubernetes/cluster-init.yaml | 2 +- cloud/kubernetes/cockroachdb-statefulset-secure.yaml | 2 +- cloud/kubernetes/cockroachdb-statefulset.yaml | 2 +- cloud/kubernetes/multiregion/client-secure.yaml | 2 +- cloud/kubernetes/multiregion/cluster-init-secure.yaml | 2 +- .../kubernetes/multiregion/cockroachdb-statefulset-secure.yaml | 2 +- .../multiregion/eks/cockroachdb-statefulset-secure-eks.yaml | 2 +- .../kubernetes/performance/cockroachdb-daemonset-insecure.yaml | 2 +- cloud/kubernetes/performance/cockroachdb-daemonset-secure.yaml | 2 +- .../performance/cockroachdb-statefulset-insecure.yaml | 2 +- .../kubernetes/performance/cockroachdb-statefulset-secure.yaml | 2 +- cloud/kubernetes/v1.6/client-secure.yaml | 2 +- cloud/kubernetes/v1.6/cluster-init-secure.yaml | 2 +- cloud/kubernetes/v1.6/cluster-init.yaml | 2 +- cloud/kubernetes/v1.6/cockroachdb-statefulset-secure.yaml | 2 +- cloud/kubernetes/v1.6/cockroachdb-statefulset.yaml | 2 +- cloud/kubernetes/v1.7/client-secure.yaml | 2 +- cloud/kubernetes/v1.7/cluster-init-secure.yaml | 2 +- cloud/kubernetes/v1.7/cluster-init.yaml | 2 +- cloud/kubernetes/v1.7/cockroachdb-statefulset-secure.yaml | 2 +- cloud/kubernetes/v1.7/cockroachdb-statefulset.yaml | 2 +- 25 files changed, 25 insertions(+), 25 deletions(-) diff --git a/cloud/kubernetes/bring-your-own-certs/client.yaml b/cloud/kubernetes/bring-your-own-certs/client.yaml index 15c00a5d6dc6..480548a61e69 100644 --- a/cloud/kubernetes/bring-your-own-certs/client.yaml +++ b/cloud/kubernetes/bring-your-own-certs/client.yaml @@ -20,7 +20,7 @@ spec: serviceAccountName: cockroachdb containers: - name: cockroachdb-client - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 # Keep a pod open indefinitely so kubectl exec can be used to get a shell to it # and run cockroach client commands, such as cockroach sql, cockroach node status, etc. command: diff --git a/cloud/kubernetes/bring-your-own-certs/cockroachdb-statefulset.yaml b/cloud/kubernetes/bring-your-own-certs/cockroachdb-statefulset.yaml index fc171baea1a9..0135429ea620 100644 --- a/cloud/kubernetes/bring-your-own-certs/cockroachdb-statefulset.yaml +++ b/cloud/kubernetes/bring-your-own-certs/cockroachdb-statefulset.yaml @@ -153,7 +153,7 @@ spec: topologyKey: kubernetes.io/hostname containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent # TODO: Change these to appropriate values for the hardware that you're running. You can see # the resources that can be allocated on each of your Kubernetes nodes by running: diff --git a/cloud/kubernetes/client-secure.yaml b/cloud/kubernetes/client-secure.yaml index fff1180cb00d..a5757abd4903 100644 --- a/cloud/kubernetes/client-secure.yaml +++ b/cloud/kubernetes/client-secure.yaml @@ -32,7 +32,7 @@ spec: mountPath: /cockroach-certs containers: - name: cockroachdb-client - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent volumeMounts: - name: client-certs diff --git a/cloud/kubernetes/cluster-init-secure.yaml b/cloud/kubernetes/cluster-init-secure.yaml index e467f1e10529..24a72ac494be 100644 --- a/cloud/kubernetes/cluster-init-secure.yaml +++ b/cloud/kubernetes/cluster-init-secure.yaml @@ -34,7 +34,7 @@ spec: mountPath: /cockroach-certs containers: - name: cluster-init - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent volumeMounts: - name: client-certs diff --git a/cloud/kubernetes/cluster-init.yaml b/cloud/kubernetes/cluster-init.yaml index 54401e24916d..02570287382b 100644 --- a/cloud/kubernetes/cluster-init.yaml +++ b/cloud/kubernetes/cluster-init.yaml @@ -10,7 +10,7 @@ spec: spec: containers: - name: cluster-init - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent command: - "/cockroach/cockroach" diff --git a/cloud/kubernetes/cockroachdb-statefulset-secure.yaml b/cloud/kubernetes/cockroachdb-statefulset-secure.yaml index 78c3f4ea06e9..a7ab271a42fd 100644 --- a/cloud/kubernetes/cockroachdb-statefulset-secure.yaml +++ b/cloud/kubernetes/cockroachdb-statefulset-secure.yaml @@ -195,7 +195,7 @@ spec: topologyKey: kubernetes.io/hostname containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent # TODO: Change these to appropriate values for the hardware that you're running. You can see # the resources that can be allocated on each of your Kubernetes nodes by running: diff --git a/cloud/kubernetes/cockroachdb-statefulset.yaml b/cloud/kubernetes/cockroachdb-statefulset.yaml index f2b53685bc8e..7c3f2a6b27d1 100644 --- a/cloud/kubernetes/cockroachdb-statefulset.yaml +++ b/cloud/kubernetes/cockroachdb-statefulset.yaml @@ -98,7 +98,7 @@ spec: topologyKey: kubernetes.io/hostname containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent # TODO: Change these to appropriate values for the hardware that you're running. You can see # the resources that can be allocated on each of your Kubernetes nodes by running: diff --git a/cloud/kubernetes/multiregion/client-secure.yaml b/cloud/kubernetes/multiregion/client-secure.yaml index 0f551981f64c..2bdf12cfb6a2 100644 --- a/cloud/kubernetes/multiregion/client-secure.yaml +++ b/cloud/kubernetes/multiregion/client-secure.yaml @@ -9,7 +9,7 @@ spec: serviceAccountName: cockroachdb containers: - name: cockroachdb-client - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent volumeMounts: - name: client-certs diff --git a/cloud/kubernetes/multiregion/cluster-init-secure.yaml b/cloud/kubernetes/multiregion/cluster-init-secure.yaml index 1b40ca6c912e..3ffa037568ef 100644 --- a/cloud/kubernetes/multiregion/cluster-init-secure.yaml +++ b/cloud/kubernetes/multiregion/cluster-init-secure.yaml @@ -11,7 +11,7 @@ spec: serviceAccountName: cockroachdb containers: - name: cluster-init - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent volumeMounts: - name: client-certs diff --git a/cloud/kubernetes/multiregion/cockroachdb-statefulset-secure.yaml b/cloud/kubernetes/multiregion/cockroachdb-statefulset-secure.yaml index c5d285a54566..c007e0621fd2 100644 --- a/cloud/kubernetes/multiregion/cockroachdb-statefulset-secure.yaml +++ b/cloud/kubernetes/multiregion/cockroachdb-statefulset-secure.yaml @@ -167,7 +167,7 @@ spec: topologyKey: kubernetes.io/hostname containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent ports: - containerPort: 26257 diff --git a/cloud/kubernetes/multiregion/eks/cockroachdb-statefulset-secure-eks.yaml b/cloud/kubernetes/multiregion/eks/cockroachdb-statefulset-secure-eks.yaml index 222de1614a4c..0766405e9134 100644 --- a/cloud/kubernetes/multiregion/eks/cockroachdb-statefulset-secure-eks.yaml +++ b/cloud/kubernetes/multiregion/eks/cockroachdb-statefulset-secure-eks.yaml @@ -185,7 +185,7 @@ spec: name: cockroach-env containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent # TODO: Change these to appropriate values for the hardware that you're running. You can see # the resources that can be allocated on each of your Kubernetes nodes by running: diff --git a/cloud/kubernetes/performance/cockroachdb-daemonset-insecure.yaml b/cloud/kubernetes/performance/cockroachdb-daemonset-insecure.yaml index 4e5e4bb45223..de690998bb5f 100644 --- a/cloud/kubernetes/performance/cockroachdb-daemonset-insecure.yaml +++ b/cloud/kubernetes/performance/cockroachdb-daemonset-insecure.yaml @@ -82,7 +82,7 @@ spec: hostNetwork: true containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent # TODO: If you configured taints to give CockroachDB exclusive access to nodes, feel free # to remove the requests and limits sections. If you didn't, you'll need to change these to diff --git a/cloud/kubernetes/performance/cockroachdb-daemonset-secure.yaml b/cloud/kubernetes/performance/cockroachdb-daemonset-secure.yaml index 392360e772f5..29741b3508d7 100644 --- a/cloud/kubernetes/performance/cockroachdb-daemonset-secure.yaml +++ b/cloud/kubernetes/performance/cockroachdb-daemonset-secure.yaml @@ -198,7 +198,7 @@ spec: topologyKey: kubernetes.io/hostname containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent # TODO: If you configured taints to give CockroachDB exclusive access to nodes, feel free # to remove the requests and limits sections. If you didn't, you'll need to change these to diff --git a/cloud/kubernetes/performance/cockroachdb-statefulset-insecure.yaml b/cloud/kubernetes/performance/cockroachdb-statefulset-insecure.yaml index 8a208d0d9af0..3b91e19d860a 100644 --- a/cloud/kubernetes/performance/cockroachdb-statefulset-insecure.yaml +++ b/cloud/kubernetes/performance/cockroachdb-statefulset-insecure.yaml @@ -141,7 +141,7 @@ spec: - name: cockroachdb # NOTE: Always use the most recent version of CockroachDB for the best # performance and reliability. - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent # TODO: Change these to appropriate values for the hardware that you're running. You can see # the resources that can be allocated on each of your Kubernetes nodes by running: diff --git a/cloud/kubernetes/performance/cockroachdb-statefulset-secure.yaml b/cloud/kubernetes/performance/cockroachdb-statefulset-secure.yaml index d4204872a3d1..3214ec2cdaed 100644 --- a/cloud/kubernetes/performance/cockroachdb-statefulset-secure.yaml +++ b/cloud/kubernetes/performance/cockroachdb-statefulset-secure.yaml @@ -232,7 +232,7 @@ spec: - name: cockroachdb # NOTE: Always use the most recent version of CockroachDB for the best # performance and reliability. - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent # TODO: Change these to appropriate values for the hardware that you're running. You can see # the resources that can be allocated on each of your Kubernetes nodes by running: diff --git a/cloud/kubernetes/v1.6/client-secure.yaml b/cloud/kubernetes/v1.6/client-secure.yaml index cccbcab45da5..429a7bf2a243 100644 --- a/cloud/kubernetes/v1.6/client-secure.yaml +++ b/cloud/kubernetes/v1.6/client-secure.yaml @@ -32,7 +32,7 @@ spec: mountPath: /cockroach-certs containers: - name: cockroachdb-client - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent volumeMounts: - name: client-certs diff --git a/cloud/kubernetes/v1.6/cluster-init-secure.yaml b/cloud/kubernetes/v1.6/cluster-init-secure.yaml index d760d413004f..904c98b813c0 100644 --- a/cloud/kubernetes/v1.6/cluster-init-secure.yaml +++ b/cloud/kubernetes/v1.6/cluster-init-secure.yaml @@ -34,7 +34,7 @@ spec: mountPath: /cockroach-certs containers: - name: cluster-init - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent volumeMounts: - name: client-certs diff --git a/cloud/kubernetes/v1.6/cluster-init.yaml b/cloud/kubernetes/v1.6/cluster-init.yaml index 6f794e21aab4..00560716e0cd 100644 --- a/cloud/kubernetes/v1.6/cluster-init.yaml +++ b/cloud/kubernetes/v1.6/cluster-init.yaml @@ -10,7 +10,7 @@ spec: spec: containers: - name: cluster-init - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent command: - "/cockroach/cockroach" diff --git a/cloud/kubernetes/v1.6/cockroachdb-statefulset-secure.yaml b/cloud/kubernetes/v1.6/cockroachdb-statefulset-secure.yaml index 00677e0870df..3cf447880d08 100644 --- a/cloud/kubernetes/v1.6/cockroachdb-statefulset-secure.yaml +++ b/cloud/kubernetes/v1.6/cockroachdb-statefulset-secure.yaml @@ -178,7 +178,7 @@ spec: topologyKey: kubernetes.io/hostname containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent ports: - containerPort: 26257 diff --git a/cloud/kubernetes/v1.6/cockroachdb-statefulset.yaml b/cloud/kubernetes/v1.6/cockroachdb-statefulset.yaml index cd411c82d8cb..bc42de88c79a 100644 --- a/cloud/kubernetes/v1.6/cockroachdb-statefulset.yaml +++ b/cloud/kubernetes/v1.6/cockroachdb-statefulset.yaml @@ -81,7 +81,7 @@ spec: topologyKey: kubernetes.io/hostname containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent ports: - containerPort: 26257 diff --git a/cloud/kubernetes/v1.7/client-secure.yaml b/cloud/kubernetes/v1.7/client-secure.yaml index 02f21a4522e0..40e8dd483bf6 100644 --- a/cloud/kubernetes/v1.7/client-secure.yaml +++ b/cloud/kubernetes/v1.7/client-secure.yaml @@ -32,7 +32,7 @@ spec: mountPath: /cockroach-certs containers: - name: cockroachdb-client - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent volumeMounts: - name: client-certs diff --git a/cloud/kubernetes/v1.7/cluster-init-secure.yaml b/cloud/kubernetes/v1.7/cluster-init-secure.yaml index 1186c863f707..b4fec679a0d1 100644 --- a/cloud/kubernetes/v1.7/cluster-init-secure.yaml +++ b/cloud/kubernetes/v1.7/cluster-init-secure.yaml @@ -34,7 +34,7 @@ spec: mountPath: /cockroach-certs containers: - name: cluster-init - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent volumeMounts: - name: client-certs diff --git a/cloud/kubernetes/v1.7/cluster-init.yaml b/cloud/kubernetes/v1.7/cluster-init.yaml index d43533d72d58..34bbb6ccae68 100644 --- a/cloud/kubernetes/v1.7/cluster-init.yaml +++ b/cloud/kubernetes/v1.7/cluster-init.yaml @@ -10,7 +10,7 @@ spec: spec: containers: - name: cluster-init - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent command: - "/cockroach/cockroach" diff --git a/cloud/kubernetes/v1.7/cockroachdb-statefulset-secure.yaml b/cloud/kubernetes/v1.7/cockroachdb-statefulset-secure.yaml index 2b10758a760c..620cb33bf65c 100644 --- a/cloud/kubernetes/v1.7/cockroachdb-statefulset-secure.yaml +++ b/cloud/kubernetes/v1.7/cockroachdb-statefulset-secure.yaml @@ -190,7 +190,7 @@ spec: topologyKey: kubernetes.io/hostname containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent ports: - containerPort: 26257 diff --git a/cloud/kubernetes/v1.7/cockroachdb-statefulset.yaml b/cloud/kubernetes/v1.7/cockroachdb-statefulset.yaml index cac92c7b7e57..ec95264678e0 100644 --- a/cloud/kubernetes/v1.7/cockroachdb-statefulset.yaml +++ b/cloud/kubernetes/v1.7/cockroachdb-statefulset.yaml @@ -93,7 +93,7 @@ spec: topologyKey: kubernetes.io/hostname containers: - name: cockroachdb - image: cockroachdb/cockroach:v24.3.2 + image: cockroachdb/cockroach:v24.3.3 imagePullPolicy: IfNotPresent ports: - containerPort: 26257 From 0ad877b389dbfd2868f8e77e972914a3b21fbdd1 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Wed, 8 Jan 2025 16:57:41 -0500 Subject: [PATCH 066/126] catalog: avoid fetching back references on non-leased descriptors Previously, when fetching immutable non-leased descriptors we still did back reference validation. This could be problematic for certain CREATE statements, since they fetch non-leased schema descriptors for name resolutions, which in turns fetches every function descriptor. To avoid this, this patch adds a level parameter to GetReferencedDescIDs, which allows the caller to filter out back references. It also update the validation code to only use mutable validation on mutable descriptors. Fixes: #138384 Release note: None --- pkg/sql/catalog/dbdesc/database_desc.go | 4 +++- pkg/sql/catalog/descriptor.go | 2 +- pkg/sql/catalog/descs/descriptor.go | 8 ++++---- pkg/sql/catalog/funcdesc/func_desc.go | 4 +++- pkg/sql/catalog/internal/validate/validate.go | 7 +++++-- pkg/sql/catalog/schemadesc/schema_desc.go | 15 +++++++++++---- .../catalog/schemadesc/synthetic_schema_desc.go | 2 +- pkg/sql/catalog/tabledesc/validate.go | 4 +++- .../typedesc/table_implicit_record_type.go | 4 +++- pkg/sql/catalog/typedesc/type_desc.go | 4 +++- .../logictest/testdata/logic_test/schema_repair | 8 +++++++- pkg/sql/testdata/telemetry/error | 2 +- pkg/sql/tests/repair_test.go | 6 ++++-- 13 files changed, 49 insertions(+), 21 deletions(-) diff --git a/pkg/sql/catalog/dbdesc/database_desc.go b/pkg/sql/catalog/dbdesc/database_desc.go index 9918a0e81143..0b67258cad7b 100644 --- a/pkg/sql/catalog/dbdesc/database_desc.go +++ b/pkg/sql/catalog/dbdesc/database_desc.go @@ -318,7 +318,9 @@ func (desc *immutable) maybeValidateSystemDatabaseSchemaVersion( // GetReferencedDescIDs returns the IDs of all descriptors referenced by // this descriptor, including itself. -func (desc *immutable) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) { +func (desc *immutable) GetReferencedDescIDs( + catalog.ValidationLevel, +) (catalog.DescriptorIDSet, error) { ids := catalog.MakeDescriptorIDSet(desc.GetID()) if desc.IsMultiRegion() { id, err := desc.MultiRegionEnumID() diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index c27b9161c1d0..20ceabea115a 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -243,7 +243,7 @@ type Descriptor interface { // GetReferencedDescIDs returns the IDs of all descriptors directly referenced // by this descriptor, including itself. - GetReferencedDescIDs() (DescriptorIDSet, error) + GetReferencedDescIDs(level ValidationLevel) (DescriptorIDSet, error) // ValidateSelf checks the internal consistency of the descriptor. ValidateSelf(vea ValidationErrorAccumulator) diff --git a/pkg/sql/catalog/descs/descriptor.go b/pkg/sql/catalog/descs/descriptor.go index 5ce20f38d7a8..c0bf46cc3617 100644 --- a/pkg/sql/catalog/descs/descriptor.go +++ b/pkg/sql/catalog/descs/descriptor.go @@ -633,6 +633,7 @@ func (tc *Collection) finalizeDescriptors( descs []catalog.Descriptor, validationLevels []catalog.ValidationLevel, ) error { + var requiredLevel catalog.ValidationLevel // Add the descriptors to the uncommitted layer if we want them to be mutable. if flags.isMutable { for i, desc := range descs { @@ -642,15 +643,14 @@ func (tc *Collection) finalizeDescriptors( } descs[i] = mut } + requiredLevel = validate.MutableRead + } else { + requiredLevel = validate.ImmutableRead } // Ensure that all descriptors are sufficiently validated. if !tc.validationModeProvider.ValidateDescriptorsOnRead() { return nil } - requiredLevel := validate.MutableRead - if !flags.layerFilters.withoutLeased { - requiredLevel = validate.ImmutableRead - } var toValidate []catalog.Descriptor for i := range descs { if validationLevels[i] < requiredLevel { diff --git a/pkg/sql/catalog/funcdesc/func_desc.go b/pkg/sql/catalog/funcdesc/func_desc.go index 5d4cfc4dfa9a..02cfce100fea 100644 --- a/pkg/sql/catalog/funcdesc/func_desc.go +++ b/pkg/sql/catalog/funcdesc/func_desc.go @@ -161,7 +161,9 @@ func (desc *immutable) NewBuilder() catalog.DescriptorBuilder { } // GetReferencedDescIDs implements the catalog.Descriptor interface. -func (desc *immutable) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) { +func (desc *immutable) GetReferencedDescIDs( + catalog.ValidationLevel, +) (catalog.DescriptorIDSet, error) { ret := catalog.MakeDescriptorIDSet(desc.GetID(), desc.GetParentID(), desc.GetParentSchemaID()) for _, id := range desc.DependsOn { ret.Add(id) diff --git a/pkg/sql/catalog/internal/validate/validate.go b/pkg/sql/catalog/internal/validate/validate.go index f56cf247ad10..1859077925e3 100644 --- a/pkg/sql/catalog/internal/validate/validate.go +++ b/pkg/sql/catalog/internal/validate/validate.go @@ -80,7 +80,7 @@ func Validate( // Collect descriptors referenced by the validated descriptors. // These are their immediate neighbors in the reference graph, and in some // special cases those neighbors' immediate neighbors also. - vdg, descGetterErr := collectDescriptorsForValidation(ctx, vd, version, descriptors) + vdg, descGetterErr := collectDescriptorsForValidation(ctx, targetLevel, vd, version, descriptors) if descGetterErr != nil { vea.reportDescGetterError(collectingReferencedDescriptors, descGetterErr) return vea.errors @@ -381,12 +381,13 @@ func (vdg *validationDescGetterImpl) addNamespaceEntries( type collectorState struct { vdg validationDescGetterImpl referencedBy catalog.DescriptorIDSet + level catalog.ValidationLevel } // addDirectReferences adds all immediate neighbors of desc to the state. func (cs *collectorState) addDirectReferences(desc catalog.Descriptor) error { cs.vdg.descriptors[desc.GetID()] = desc - idSet, err := desc.GetReferencedDescIDs() + idSet, err := desc.GetReferencedDescIDs(cs.level) if err != nil { return err } @@ -432,6 +433,7 @@ func (cs *collectorState) getMissingDescs( // possible descriptors required for validation. func collectDescriptorsForValidation( ctx context.Context, + level catalog.ValidationLevel, vd ValidationDereferencer, version clusterversion.ClusterVersion, descriptors []catalog.Descriptor, @@ -442,6 +444,7 @@ func collectDescriptorsForValidation( namespace: make(map[descpb.NameInfo]descpb.ID, len(descriptors)), }, referencedBy: catalog.MakeDescriptorIDSet(), + level: level, } for _, desc := range descriptors { if desc == nil || desc.Dropped() { diff --git a/pkg/sql/catalog/schemadesc/schema_desc.go b/pkg/sql/catalog/schemadesc/schema_desc.go index 631d62a198c2..5b0f2540236f 100644 --- a/pkg/sql/catalog/schemadesc/schema_desc.go +++ b/pkg/sql/catalog/schemadesc/schema_desc.go @@ -270,13 +270,20 @@ func (desc *immutable) ValidateSelf(vea catalog.ValidationErrorAccumulator) { // GetReferencedDescIDs returns the IDs of all descriptors referenced by // this descriptor, including itself. -func (desc *immutable) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) { +func (desc *immutable) GetReferencedDescIDs( + level catalog.ValidationLevel, +) (catalog.DescriptorIDSet, error) { ret := catalog.MakeDescriptorIDSet(desc.GetID(), desc.GetParentID()) - for _, f := range desc.Functions { - for _, sig := range f.Signatures { - ret.Add(sig.ID) + // We only need to resolve functions in this schema if we are validating + // back references as well. + if level&catalog.ValidationLevelBackReferences == catalog.ValidationLevelBackReferences { + for _, f := range desc.Functions { + for _, sig := range f.Signatures { + ret.Add(sig.ID) + } } } + return ret, nil } diff --git a/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go b/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go index 8034c1a1e660..bd5f2ebb4862 100644 --- a/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go +++ b/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go @@ -76,7 +76,7 @@ func (p synthetic) NewBuilder() catalog.DescriptorBuilder { "%s schema cannot create a builder", p.kindName()) return nil // unreachable } -func (p synthetic) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) { +func (p synthetic) GetReferencedDescIDs(catalog.ValidationLevel) (catalog.DescriptorIDSet, error) { return catalog.DescriptorIDSet{}, nil } func (p synthetic) ValidateSelf(_ catalog.ValidationErrorAccumulator) { diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 0ff7d7fb9e8a..7763d8ab7850 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -58,7 +58,9 @@ func (desc *wrapper) ValidateTxnCommit( // GetReferencedDescIDs returns the IDs of all descriptors referenced by // this descriptor, including itself. -func (desc *wrapper) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) { +func (desc *wrapper) GetReferencedDescIDs( + catalog.ValidationLevel, +) (catalog.DescriptorIDSet, error) { ids := catalog.MakeDescriptorIDSet(desc.GetID(), desc.GetParentID()) // TODO(richardjcai): Remove logic for keys.PublicSchemaID in 22.2. if desc.GetParentSchemaID() != keys.PublicSchemaID { diff --git a/pkg/sql/catalog/typedesc/table_implicit_record_type.go b/pkg/sql/catalog/typedesc/table_implicit_record_type.go index 5b5f6dbc0cea..2451c1485908 100644 --- a/pkg/sql/catalog/typedesc/table_implicit_record_type.go +++ b/pkg/sql/catalog/typedesc/table_implicit_record_type.go @@ -150,7 +150,9 @@ func (v *tableImplicitRecordType) NewBuilder() catalog.DescriptorBuilder { } // GetReferencedDescIDs implements the catalog.Descriptor interface. -func (v *tableImplicitRecordType) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) { +func (v *tableImplicitRecordType) GetReferencedDescIDs( + catalog.ValidationLevel, +) (catalog.DescriptorIDSet, error) { return catalog.DescriptorIDSet{}, errors.AssertionFailedf( "GetReferencedDescIDs are unsupported for implicit table record types") } diff --git a/pkg/sql/catalog/typedesc/type_desc.go b/pkg/sql/catalog/typedesc/type_desc.go index 5802565e7a74..0d49544a2446 100644 --- a/pkg/sql/catalog/typedesc/type_desc.go +++ b/pkg/sql/catalog/typedesc/type_desc.go @@ -551,7 +551,9 @@ func (desc *immutable) validateEnumMembers(vea catalog.ValidationErrorAccumulato // GetReferencedDescIDs returns the IDs of all descriptors referenced by // this descriptor, including itself. -func (desc *immutable) GetReferencedDescIDs() (catalog.DescriptorIDSet, error) { +func (desc *immutable) GetReferencedDescIDs( + catalog.ValidationLevel, +) (catalog.DescriptorIDSet, error) { ids := catalog.MakeDescriptorIDSet(desc.GetReferencingDescriptorIDs()...) ids.Add(desc.GetParentID()) // TODO(richardjcai): Remove logic for keys.PublicSchemaID in 22.2. diff --git a/pkg/sql/logictest/testdata/logic_test/schema_repair b/pkg/sql/logictest/testdata/logic_test/schema_repair index 631379024b5c..9997f70b2274 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema_repair +++ b/pkg/sql/logictest/testdata/logic_test/schema_repair @@ -217,7 +217,7 @@ SELECT * FROM corrupt_backref_fk ---- 1 a -statement error invalid foreign key backreference +statement error (invalid foreign key backreference)*(.*referenced descriptor not found.*)* DROP TABLE corrupt_backref_fk statement ok @@ -241,9 +241,15 @@ SELECT * FROM corrupt_backref_view ---- 1 a +onlyif config local-legacy-schema-changer + statement error pgcode XX000 invalid depended-on-by relation back reference DROP TABLE corrupt_backref_view +skipif config local-legacy-schema-changer +statement error pgcode XXUUU .*descriptor not found.* +DROP TABLE corrupt_backref_view + statement ok CREATE TYPE corrupt_backref_typ AS ENUM ('a', 'b'); CREATE TABLE corrupt_typ (k INT PRIMARY KEY, v corrupt_backref_typ); diff --git a/pkg/sql/testdata/telemetry/error b/pkg/sql/testdata/telemetry/error index 4ce58e39310b..77c8a488d41e 100644 --- a/pkg/sql/testdata/telemetry/error +++ b/pkg/sql/testdata/telemetry/error @@ -43,7 +43,7 @@ SELECT crdb_internal.unsafe_upsert_descriptor(id, crdb_internal.json_to_pb('desc feature-usage DROP TABLE tbl CASCADE; ---- -error: pq: internal error: building declarative schema change targets for DROP TABLE: relation "tbl" (...): missing fk back reference "tbl_customer_fkey" to "tbl" from "fktbl" +error: pq: internal error: executing declarative schema change StatementPhase stage 1 of 1 with 46 MutationType ops (rollback=false) for DROP TABLE: error executing StatementPhase stage 1 of 1 with 46 MutationType ops: *scop.MakePublicForeignKeyConstraintValidated: &{{{}} 105 2}: relation "tbl" (...): missing fk back reference "tbl_customer_fkey" to "tbl" from "fktbl" errorcodes.XX000 sql.schema.validation_errors.read.backward_references.relation diff --git a/pkg/sql/tests/repair_test.go b/pkg/sql/tests/repair_test.go index 83ef9942dffc..c15cac847098 100644 --- a/pkg/sql/tests/repair_test.go +++ b/pkg/sql/tests/repair_test.go @@ -906,8 +906,10 @@ func TestCorruptDescriptorRepair(t *testing.T) { tdb.CheckQueryResults(t, `SELECT * FROM testdb.parent`, [][]string{{"1", "a"}}) // Dropping the table should fail, because the table descriptor will fail - // the validation checks when being read from storage. - tdb.ExpectErr(t, "invalid foreign key backreference", `DROP TABLE testdb.parent`) + // the validation checks. For the declarative schema changer before the + // execution phase the sel validation will fail with a generic error because + // all reads are immutable. + tdb.ExpectErr(t, "referenced descriptor ID 107: looking up ID 107: descriptor not found", `DROP TABLE testdb.parent`) const parentVersion = `SELECT crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', sd.descriptor, false)->'table'->>'version' From f2d483ed80d4d4974a7641cd0a06f2e73d1324d0 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Wed, 8 Jan 2025 17:00:40 -0500 Subject: [PATCH 067/126] sql: avoid fetching schema descriptor as mutable Previously, the logic detect schema changes on schema descriptors during CREATE statements incorrectly fetched a mutable descriptor. To address this, this patch only bypasses leasing for this read operation, so that we don't run the full back reference validation during create operations on schema. This patch also adds tests to make sure concurrent CREATE statements never hit unexpected contention. Release note: None --- pkg/sql/create_table.go | 2 +- pkg/sql/schemachanger/schemachanger_test.go | 150 ++++++++++++++++++++ 2 files changed, 151 insertions(+), 1 deletion(-) diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 4e58d9390e62..cbcd043e7322 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -91,7 +91,7 @@ func (p *planner) getNonTemporarySchemaForCreate( case catalog.SchemaPublic: return sc, nil case catalog.SchemaUserDefined: - sc, err := p.Descriptors().MutableByID(p.txn).Schema(ctx, sc.GetID()) + sc, err = p.Descriptors().ByIDWithoutLeased(p.Txn()).Get().Schema(ctx, sc.GetID()) if err != nil { return nil, err } diff --git a/pkg/sql/schemachanger/schemachanger_test.go b/pkg/sql/schemachanger/schemachanger_test.go index a686f2519278..01454eedaa68 100644 --- a/pkg/sql/schemachanger/schemachanger_test.go +++ b/pkg/sql/schemachanger/schemachanger_test.go @@ -957,3 +957,153 @@ func TestSchemaChangerFailsOnMissingDesc(t *testing.T) { tdb.CheckQueryResults(t, "SELECT status FROM crdb_internal.jobs WHERE statement LIKE '%ADD COLUMN%'", [][]string{{"failed"}}) } + +// TestCreateObjectConcurrency validates that concurrent create object with +// independent references never hit txn retry errors. All objects are created +// under the same schema. +func TestCreateObjectConcurrency(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Validate concurrency behaviour for objects under a schema + tests := []struct { + name string + setupStmt string + firstStmt string + secondStmt string + }{ + { + name: "create table with function references", + setupStmt: ` +CREATE FUNCTION public.fn1 (input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT input::INT8; + $$; +CREATE FUNCTION public.wrap(input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT public.fn1(input); + $$; +CREATE FUNCTION public.wrap2(input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT public.fn1(input); + $$; +`, + firstStmt: "CREATE TABLE t1(n int default public.wrap(10))", + secondStmt: "CREATE TABLE t2(n int default public.wrap2(10))", + }, + { + name: "create table with type reference", + setupStmt: ` +CREATE TYPE status AS ENUM ('open', 'closed', 'inactive'); +CREATE TYPE status1 AS ENUM ('open', 'closed', 'inactive'); +`, + firstStmt: "CREATE TABLE t1(n status)", + secondStmt: "CREATE TABLE t2(n status1)", + }, + { + name: "create view with type references", + setupStmt: ` +CREATE TYPE status AS ENUM ('open', 'closed', 'inactive'); +CREATE TYPE status1 AS ENUM ('open', 'closed', 'inactive'); +CREATE FUNCTION public.fn1 (input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT input::INT8; + $$; +CREATE FUNCTION public.wrap(input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT public.fn1(input); + $$; +CREATE FUNCTION public.wrap2(input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT public.fn1(input); + $$; +CREATE TABLE t1(n int default public.wrap(10)); +CREATE TABLE t2(n int default public.wrap2(10)); +`, + // Note: Views cannot invoke UDFs directly yet. + firstStmt: "CREATE VIEW v1 AS (SELECT n, 'open'::status FROM public.t1)", + secondStmt: "CREATE VIEW v2 AS (SELECT n, 'open'::status1 FROM public.t2)", + }, + { + name: "create sequence with ownership", + setupStmt: ` +CREATE TABLE t1(n int); +CREATE TABLE t2(n int); +`, + firstStmt: "CREATE SEQUENCE sq1 OWNED BY t1.n", + secondStmt: "CREATE SEQUENCE sq2 OWNED BY t2.n", + }, + { + name: "create type", + firstStmt: "CREATE TYPE status AS ENUM ('open', 'closed', 'inactive');", + secondStmt: "CREATE TYPE status1 AS ENUM ('open', 'closed', 'inactive');", + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + ctx := context.Background() + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ + // This would work with secondary tenants as well, but the span config + // limited logic can hit transaction retries on the span_count table. + DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(138733), + }) + defer s.Stopper().Stop(ctx) + + runner := sqlutils.MakeSQLRunner(sqlDB) + firstConn, err := sqlDB.Conn(ctx) + require.NoError(t, err) + defer func() { + require.NoError(t, firstConn.Close()) + }() + secondConn, err := sqlDB.Conn(ctx) + require.NoError(t, err) + defer func() { + require.NoError(t, secondConn.Close()) + }() + + firstConnReady := make(chan struct{}) + secondConnReady := make(chan struct{}) + + runner.Exec(t, test.setupStmt) + + grp := ctxgroup.WithContext(ctx) + + grp.Go(func() error { + defer close(firstConnReady) + tx, err := firstConn.BeginTx(ctx, nil) + if err != nil { + return err + } + _, err = tx.Exec(test.firstStmt) + if err != nil { + return err + } + firstConnReady <- struct{}{} + <-secondConnReady + return tx.Commit() + }) + grp.Go(func() error { + defer close(secondConnReady) + tx, err := secondConn.BeginTx(ctx, nil) + if err != nil { + return err + } + _, err = tx.Exec(test.secondStmt) + if err != nil { + return err + } + <-firstConnReady + secondConnReady <- struct{}{} + return tx.Commit() + }) + require.NoError(t, grp.Wait()) + }) + } +} From 09f4c9ced099fbdecb46f7663ed4de52e3074ab6 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Mon, 13 Jan 2025 13:54:55 -0500 Subject: [PATCH 068/126] roachtest: mark ORM tests that rely on DO syntax as passing Since https://github.com/cockroachdb/cockroach/pull/138709 was merged, more tests began passing. Release note: None --- pkg/cmd/roachtest/tests/asyncpg_blocklist.go | 3 --- pkg/cmd/roachtest/tests/gopg_blocklist.go | 1 - pkg/cmd/roachtest/tests/libpq_blocklist.go | 2 -- pkg/cmd/roachtest/tests/npgsql_blocklist.go | 1 + pkg/cmd/roachtest/tests/rust_postgres_blocklist.go | 1 - 5 files changed, 1 insertion(+), 7 deletions(-) diff --git a/pkg/cmd/roachtest/tests/asyncpg_blocklist.go b/pkg/cmd/roachtest/tests/asyncpg_blocklist.go index 9a0cc6ff4114..a5b98c25a2da 100644 --- a/pkg/cmd/roachtest/tests/asyncpg_blocklist.go +++ b/pkg/cmd/roachtest/tests/asyncpg_blocklist.go @@ -52,14 +52,11 @@ var asyncpgBlocklist = blocklist{ `test_listeners.TestListeners.test_listen_notletters`: "LISTEN - https://github.com/cockroachdb/cockroach/issues/41522", `test_listeners.TestLogListeners.test_log_listener_01`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511", `test_listeners.TestLogListeners.test_log_listener_02`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511", - `test_listeners.TestLogListeners.test_log_listener_03`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511", `test_pool.TestPool.test_pool_remote_close`: "unsupported pg_terminate_backend() function", - `test_prepare.TestPrepare.test_prepare_09_raise_error`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511", `test_prepare.TestPrepare.test_prepare_14_explain`: "unknown", `test_prepare.TestPrepare.test_prepare_16_command_result`: "unknown", `test_prepare.TestPrepare.test_prepare_19_concurrent_calls`: "unknown", `test_prepare.TestPrepare.test_prepare_28_max_args`: "unknown", - `test_prepare.TestPrepare.test_prepare_31_pgbouncer_note`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511", `test_prepare.TestPrepare.test_prepare_statement_invalid`: "experimental feature - https://github.com/cockroachdb/cockroach/issues/49329", `test_timeout.TestTimeout.test_timeout_06`: "unknown", `test_utils.TestUtils.test_mogrify_simple`: "multi-dim arrays - https://github.com/cockroachdb/cockroach/issues/32552", diff --git a/pkg/cmd/roachtest/tests/gopg_blocklist.go b/pkg/cmd/roachtest/tests/gopg_blocklist.go index 8f5fbe9f1358..4bc938c67f64 100644 --- a/pkg/cmd/roachtest/tests/gopg_blocklist.go +++ b/pkg/cmd/roachtest/tests/gopg_blocklist.go @@ -28,7 +28,6 @@ var gopgBlockList = blocklist{ "v10.ExampleDB_Model_postgresArrayStructTag": "32552", "v10.TestConversion": "32552", "v10.TestGinkgo": "41522", - "v10.TestGocheck": "17511", "v10.TestReadColumnValue": "26925", "v10.TestUnixSocket": "31113", } diff --git a/pkg/cmd/roachtest/tests/libpq_blocklist.go b/pkg/cmd/roachtest/tests/libpq_blocklist.go index c9fb0ba81663..04c484d13412 100644 --- a/pkg/cmd/roachtest/tests/libpq_blocklist.go +++ b/pkg/cmd/roachtest/tests/libpq_blocklist.go @@ -6,13 +6,11 @@ package tests var libPQBlocklist = blocklist{ - "pq.ExampleConnectorWithNoticeHandler": "unknown", "pq.TestBinaryByteSliceToInt": "41547", "pq.TestBinaryByteSlicetoUUID": "41547", "pq.TestConnListen": "41522", "pq.TestConnUnlisten": "41522", "pq.TestConnUnlistenAll": "41522", - "pq.TestConnectorWithNoticeHandler_Simple": "unknown", "pq.TestConnectorWithNotificationHandler_Simple": "unknown", "pq.TestCopyInRaiseStmtTrigger": "5807", "pq.TestCopyInTypes": "5807", diff --git a/pkg/cmd/roachtest/tests/npgsql_blocklist.go b/pkg/cmd/roachtest/tests/npgsql_blocklist.go index 18050c61fd03..cf651d527d2f 100644 --- a/pkg/cmd/roachtest/tests/npgsql_blocklist.go +++ b/pkg/cmd/roachtest/tests/npgsql_blocklist.go @@ -661,5 +661,6 @@ var npgsqlBlocklist = blocklist{ } var npgsqlIgnoreList = blocklist{ + `Npgsql.Tests.ConnectionTests(Multiplexing).Fail_connect_then_succeed(True)`: "flaky", `Npgsql.Tests.TransactionTests(Multiplexing).Failed_transaction_on_close_with_custom_timeout`: "flaky", } diff --git a/pkg/cmd/roachtest/tests/rust_postgres_blocklist.go b/pkg/cmd/roachtest/tests/rust_postgres_blocklist.go index 1c65c4eb5564..b84d1608cfa1 100644 --- a/pkg/cmd/roachtest/tests/rust_postgres_blocklist.go +++ b/pkg/cmd/roachtest/tests/rust_postgres_blocklist.go @@ -38,7 +38,6 @@ var rustPostgresBlocklist = blocklist{ "test.copy_in": "COPY FROM not supported in extended protocol", "test.copy_in_abort": "COPY FROM not supported in extended protocol", "test.nested_transactions": "default int size (int4 vs int8) mismatch", - "test.notice_callback": "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511", "test.notifications_blocking_iter": "unsupported feature - https://github.com/cockroachdb/cockroach/issues/41522", "test.notifications_iter": "unsupported feature - https://github.com/cockroachdb/cockroach/issues/41522", "test.notifications_timeout_iter": "unsupported feature - https://github.com/cockroachdb/cockroach/issues/41522", From 363ccb93e61e9479b0103e31d6fe2d07cf6154ca Mon Sep 17 00:00:00 2001 From: Ludovic Leroux Date: Mon, 13 Jan 2025 13:46:49 -0500 Subject: [PATCH 069/126] roachprod: promhelper fix project and wipe This PR fixes a bug introduced in #138711 and also adds deletion of Prometheus targets at cluster wipe. Before #138711, the GCE provider defaults were defined during init(). This logic was moved to an init function to allow the `drtprod` command to define its own defaults via environment variables. This introduced a state in which where the promhelper client's defaults for `SupportedPromProject` is initialized with `gce.DefaultProject()` before this value is initialized, and no Prometheus targets are ever pushed. This PR removes the `promhelperclient.DefaultClient` that should not be used anymore, and computing the defaults in `NewPromClient()`. This PR also delegates the checks on whether or not providers and projects are supported to the promhelperclient package to simplify the logic in the callers. Also, prior to this PR, if an `insecure` cluster was reused as a `secure` cluster during a `roachtest` run, the promhelper client would delete the `secure` configuration during cluster destruction, but would leave the `insecure` configuration (as the promhelper clients tries to delete `secure` first, then `insecure` if not found). This was creating stale Prometheus targets. This PR introduces the deletion of the Prometheus targets at cluster wipe to fix this. Epic: none Release note: None Signed-off-by: Ludovic Leroux --- pkg/roachprod/cloud/cluster_cloud.go | 68 ++++++++++++++++-------- pkg/roachprod/install/cluster_synced.go | 12 ++++- pkg/roachprod/promhelperclient/client.go | 40 +++++++++----- pkg/roachprod/roachprod.go | 43 ++++++++------- 4 files changed, 111 insertions(+), 52 deletions(-) diff --git a/pkg/roachprod/cloud/cluster_cloud.go b/pkg/roachprod/cloud/cluster_cloud.go index a3cf199a727d..f47062a21c89 100644 --- a/pkg/roachprod/cloud/cluster_cloud.go +++ b/pkg/roachprod/cloud/cluster_cloud.go @@ -473,35 +473,61 @@ func ShrinkCluster(l *logger.Logger, c *Cluster, numNodes int) error { return nil } -// DestroyCluster TODO(peter): document -func DestroyCluster(l *logger.Logger, c *Cluster) error { +func (c *Cluster) DeletePrometheusConfig(ctx context.Context, l *logger.Logger) error { + + cl := promhelperclient.NewPromClient() + stopSpinner := ui.NewDefaultSpinner(l, "Destroying Prometheus configs").Start() - // check if any node is supported as promhelper cluster + defer stopSpinner() + for _, node := range c.VMs { - if _, ok := promhelperclient.SupportedPromProjects[node.Project]; ok && - node.Provider == gce.ProviderName { - if err := promhelperclient.NewPromClient().DeleteClusterConfig(context.Background(), - c.Name, false, false /* insecure */, l); err != nil { - // TODO(bhaskar): Obtain secure cluster information. - // Cluster does not have the information on secure or not. So, we retry as insecure - // if delete fails with cluster as secure - if promhelperclient.IsNotFoundError(err) { - if err = promhelperclient.NewPromClient().DeleteClusterConfig(context.Background(), - c.Name, false, true /* insecure */, l); err != nil { - l.Errorf("Failed to delete the cluster config with cluster as insecure and secure: %v", err) - } - } else { - l.Errorf("Failed to delete the cluster config with cluster as secure: %v", err) - } + + // only gce is supported for prometheus + if !cl.IsSupportedNodeProvider(node.Provider) { + continue + } + if !cl.IsSupportedPromProject(node.Project) { + continue + } + + err := cl.DeleteClusterConfig(ctx, c.Name, false, false /* insecure */, l) + if err != nil { + + if !promhelperclient.IsNotFoundError(err) { + return errors.Wrapf( + err, + "failed to delete the cluster config with cluster as secure", + ) } - break + + // TODO(bhaskar): Obtain secure cluster information. + // Cluster does not have the information on secure or not. + // So, we retry as insecure if delete fails with cluster as secure. + if err = cl.DeleteClusterConfig(ctx, c.Name, false, true /* insecure */, l); err != nil { + return errors.Wrapf( + err, + "failed to delete the cluster config with cluster as insecure and secure", + ) + } + } + break + + } + + return nil +} + +// DestroyCluster TODO(peter): document +func DestroyCluster(l *logger.Logger, c *Cluster) error { + + if err := c.DeletePrometheusConfig(context.Background(), l); err != nil { + l.Printf("WARNING: failed to delete the prometheus config (already wiped?): %s", err) } - stopSpinner() // DNS entries are destroyed first to ensure that the GC job will not try // and clean-up entries prematurely. - stopSpinner = ui.NewDefaultSpinner(l, "Destroying DNS entries").Start() + stopSpinner := ui.NewDefaultSpinner(l, "Destroying DNS entries").Start() dnsErr := vm.FanOutDNS(c.VMs, func(p vm.DNSProvider, vms vm.List) error { return p.DeleteRecordsBySubdomain(context.Background(), c.Name) }) diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index 82a19588cb93..5a406f3d4654 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -593,7 +593,7 @@ func (c *SyncedCluster) Wipe(ctx context.Context, l *logger.Logger, preserveCert if err := c.Stop(ctx, l, int(unix.SIGKILL), true /* wait */, 0 /* gracePeriod */, ""); err != nil { return err } - return c.Parallel(ctx, l, WithNodes(c.Nodes).WithDisplay(display), func(ctx context.Context, node Node) (*RunResultDetails, error) { + err := c.Parallel(ctx, l, WithNodes(c.Nodes).WithDisplay(display), func(ctx context.Context, node Node) (*RunResultDetails, error) { var cmd string if c.IsLocal() { // Not all shells like brace expansion, so we'll do it here @@ -622,6 +622,16 @@ func (c *SyncedCluster) Wipe(ctx context.Context, l *logger.Logger, preserveCert } return c.runCmdOnSingleNode(ctx, l, node, cmd, defaultCmdOpts("wipe")) }) + if err != nil { + return err + } + + err = c.Cluster.DeletePrometheusConfig(ctx, l) + if err != nil { + l.Printf("WARNING: failed to delete the prometheus config (already wiped?): %s", err) + } + + return nil } // NodeStatus contains details about the status of a node. diff --git a/pkg/roachprod/promhelperclient/client.go b/pkg/roachprod/promhelperclient/client.go index 49426b193a3a..eb2384d89c44 100644 --- a/pkg/roachprod/promhelperclient/client.go +++ b/pkg/roachprod/promhelperclient/client.go @@ -38,9 +38,6 @@ const ( ErrorMessagePrefix = "request failed with status %d" ) -// SupportedPromProjects are the projects supported for prometheus target -var SupportedPromProjects = map[string]struct{}{gce.DefaultProject(): {}} - // The URL for the Prometheus registration service. An empty string means that the // Prometheus integration is disabled. Should be accessed through // getPrometheusRegistrationUrl(). @@ -63,12 +60,15 @@ type PromClient struct { // newTokenSource is the token generator source. newTokenSource func(ctx context.Context, audience string, opts ...idtoken.ClientOption) ( oauth2.TokenSource, error) -} -// DefaultPromClient is the default instance of PromClient. This instance should -// be used unless custom configuration is needed. -var DefaultPromClient = NewPromClient() + // supportedPromProviders are the providers supported for prometheus target + supportedPromProviders map[string]struct{} + + // supportedPromProjects are the projects supported for prometheus target + supportedPromProjects map[string]struct{} +} +// IsNotFoundError returns true if the error is a 404 error. func IsNotFoundError(err error) bool { return strings.Contains(err.Error(), fmt.Sprintf(ErrorMessagePrefix, http.StatusNotFound)) } @@ -76,11 +76,13 @@ func IsNotFoundError(err error) bool { // NewPromClient returns a new instance of PromClient func NewPromClient() *PromClient { return &PromClient{ - promUrl: promRegistrationUrl, - disabled: promRegistrationUrl == "", - httpPut: httputil.Put, - httpDelete: httputil.Delete, - newTokenSource: idtoken.NewTokenSource, + promUrl: promRegistrationUrl, + disabled: promRegistrationUrl == "", + httpPut: httputil.Put, + httpDelete: httputil.Delete, + newTokenSource: idtoken.NewTokenSource, + supportedPromProviders: map[string]struct{}{gce.ProviderName: {}}, + supportedPromProjects: map[string]struct{}{gce.DefaultProject(): {}}, } } @@ -183,6 +185,20 @@ func getUrl(promUrl, clusterName string) string { return fmt.Sprintf("%s/%s/%s/%s", promUrl, resourceVersion, resourceName, clusterName) } +// IsSupportedNodeProvider returns true if the provider is supported +// for prometheus target. +func (c *PromClient) IsSupportedNodeProvider(provider string) bool { + _, ok := c.supportedPromProviders[provider] + return ok +} + +// IsSupportedPromProject returns true if the project is supported +// for prometheus target. +func (c *PromClient) IsSupportedPromProject(project string) bool { + _, ok := c.supportedPromProjects[project] + return ok +} + // CCParams are the params for the cluster configs type CCParams struct { Targets []string `yaml:"targets"` diff --git a/pkg/roachprod/roachprod.go b/pkg/roachprod/roachprod.go index 887e49994232..3fea70042582 100644 --- a/pkg/roachprod/roachprod.go +++ b/pkg/roachprod/roachprod.go @@ -815,32 +815,39 @@ func updatePrometheusTargets( return err } + cl := promhelperclient.NewPromClient() nodeIPPorts := make(map[int]*promhelperclient.NodeInfo) nodeIPPortsMutex := syncutil.RWMutex{} var wg sync.WaitGroup for _, node := range c.Nodes { - if _, ok := promhelperclient.SupportedPromProjects[c.VMs[node-1].Project]; ok && - c.VMs[node-1].Provider == gce.ProviderName { - wg.Add(1) - go func(index int, v vm.VM) { - defer wg.Done() - // only gce is supported for prometheus - desc, err := c.DiscoverService(ctx, install.Node(index), "", install.ServiceTypeUI, 0) - if err != nil { - l.Errorf("error getting the port for node %d: %v", index, err) - return - } - nodeInfo := fmt.Sprintf("%s:%d", v.PrivateIP, desc.Port) - nodeIPPortsMutex.Lock() - // ensure atomicity in map update - nodeIPPorts[index] = &promhelperclient.NodeInfo{Target: nodeInfo, CustomLabels: createLabels(v)} - nodeIPPortsMutex.Unlock() - }(int(node), c.VMs[node-1]) + + // only gce is supported for prometheus + if !cl.IsSupportedNodeProvider(c.VMs[node-1].Provider) { + continue } + if !cl.IsSupportedPromProject(c.VMs[node-1].Project) { + continue + } + + wg.Add(1) + go func(index int, v vm.VM) { + defer wg.Done() + desc, err := c.DiscoverService(ctx, install.Node(index), "", install.ServiceTypeUI, 0) + if err != nil { + l.Errorf("error getting the port for node %d: %v", index, err) + return + } + nodeInfo := fmt.Sprintf("%s:%d", v.PrivateIP, desc.Port) + nodeIPPortsMutex.Lock() + // ensure atomicity in map update + nodeIPPorts[index] = &promhelperclient.NodeInfo{Target: nodeInfo, CustomLabels: createLabels(v)} + nodeIPPortsMutex.Unlock() + }(int(node), c.VMs[node-1]) + } wg.Wait() if len(nodeIPPorts) > 0 { - if err := promhelperclient.DefaultPromClient.UpdatePrometheusTargets(ctx, + if err := cl.UpdatePrometheusTargets(ctx, c.Name, false, nodeIPPorts, !c.Secure, l); err != nil { l.Errorf("creating cluster config failed for the ip:ports %v: %v", nodeIPPorts, err) } From 0d25b695cef60834eb299a063e35e7de309ac054 Mon Sep 17 00:00:00 2001 From: Matt Spilchen Date: Mon, 13 Jan 2025 15:17:12 -0400 Subject: [PATCH 070/126] sql/schemachanger: Add support for storing policy command and type A previous commit introduced basic support for CREATE/DROP POLICY. This commit expands on that functionality by storing additional details in the policy descriptor. Specifically, it adds support for storing the policy type (restrictive or permissive) and the policy command (ALL, SELECT, INSERT, UPDATE, or DELETE). Since neither the policy type nor the policy command will be modifiable via ALTER POLICY, these attributes are included in the Policy element within the DSC, rather than as separate elements. Epic: CRDB-11724 Informs: #136696 Release note: None --- pkg/sql/catalog/catpb/BUILD.bazel | 1 + pkg/sql/catalog/catpb/enum.proto | 29 ++++++++++ pkg/sql/catalog/catpb/redact.go | 12 ++++ pkg/sql/catalog/descpb/structured.proto | 6 ++ pkg/sql/catalog/tabledesc/validate.go | 8 +++ pkg/sql/catalog/tabledesc/validate_test.go | 56 ++++++++++++++---- .../testdata/logic_test/row_level_security | 54 ++++++++++++++++++ .../internal/scbuildstmt/create_policy.go | 34 +++++++++++ .../scbuild/testdata/create_policy | 52 +++++++++++++++++ .../scbuild/testdata/drop_policy | 57 +++++++++++++++++++ pkg/sql/schemachanger/scdecomp/decomp.go | 2 + .../scexec/scmutationexec/policy.go | 4 +- pkg/sql/schemachanger/scpb/elements.proto | 2 + pkg/sql/schemachanger/scpb/uml/table.puml | 2 + .../create_policy/create_policy.explain | 4 +- .../create_policy/create_policy.side_effects | 8 ++- .../drop_policy/drop_policy.explain | 4 +- .../drop_policy/drop_policy.side_effects | 12 ++-- .../lint/passes/redactcheck/redactcheck.go | 6 +- 19 files changed, 330 insertions(+), 23 deletions(-) create mode 100644 pkg/sql/catalog/catpb/redact.go create mode 100644 pkg/sql/schemachanger/scbuild/testdata/create_policy create mode 100644 pkg/sql/schemachanger/scbuild/testdata/drop_policy diff --git a/pkg/sql/catalog/catpb/BUILD.bazel b/pkg/sql/catalog/catpb/BUILD.bazel index 2908dd64185b..afe870fd8989 100644 --- a/pkg/sql/catalog/catpb/BUILD.bazel +++ b/pkg/sql/catalog/catpb/BUILD.bazel @@ -40,6 +40,7 @@ go_library( "job_id.go", "multiregion.go", "privilege.go", + "redact.go", ":gen-privilegedescversion-stringer", # keep ], embed = [":catpb_go_proto"], diff --git a/pkg/sql/catalog/catpb/enum.proto b/pkg/sql/catalog/catpb/enum.proto index b55d9c65f948..9f5eb9645cca 100644 --- a/pkg/sql/catalog/catpb/enum.proto +++ b/pkg/sql/catalog/catpb/enum.proto @@ -11,6 +11,10 @@ syntax = "proto3"; package cockroach.sql.catalog.catpb; option go_package = "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"; +// NOTE: When adding new enum types to this file, consider implementing the +// redact.SafeValue interface if the values are safe to display in redacted logs. +// This implementation can be added to the redact.go file in this package. + // SystemColumnKind is an enum representing the different kind of system // columns that can be synthesized by the execution engine. enum SystemColumnKind { @@ -48,3 +52,28 @@ enum InvertedIndexColumnKind { // text columns. TRIGRAM = 1; } + +// Type represents the category of policy. +// +// NOTE: When adding a new enum value, ensure it is only utilized after +// verifying that the cluster version has been finalized. Older versions +// will validate that only recognized enum values are present. +enum PolicyType { + POLICYTYPE_UNUSED = 0; + PERMISSIVE = 1; + RESTRICTIVE = 2; +} + +// PolicyCommand specifies the SQL commands to which the policy applies. +// +// NOTE: When adding a new enum value, ensure it is only utilized after +// verifying that the cluster version has been finalized. Older versions +// will validate that only recognized enum values are present. +enum PolicyCommand { + POLICYCOMMAND_UNUSED = 0; + ALL = 1; + SELECT = 2; + INSERT = 3; + UPDATE = 4; + DELETE = 5; +} diff --git a/pkg/sql/catalog/catpb/redact.go b/pkg/sql/catalog/catpb/redact.go new file mode 100644 index 000000000000..a09cf6de6b7a --- /dev/null +++ b/pkg/sql/catalog/catpb/redact.go @@ -0,0 +1,12 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package catpb + +// SafeValue implements the redact.SafeValue interface. +func (PolicyType) SafeValue() {} + +// SafeValue implements the redact.SafeValue interface. +func (PolicyCommand) SafeValue() {} diff --git a/pkg/sql/catalog/descpb/structured.proto b/pkg/sql/catalog/descpb/structured.proto index b7cebda8cd37..b27a1a2ef2ff 100644 --- a/pkg/sql/catalog/descpb/structured.proto +++ b/pkg/sql/catalog/descpb/structured.proto @@ -706,6 +706,12 @@ message PolicyDescriptor { // The name of the policy. Unique within a table, and cannot be qualified. optional string name = 2 [(gogoproto.nullable) = false]; + + // Type indicates whether the policy is permissive or restrictive. + optional cockroach.sql.catalog.catpb.PolicyType type = 3 [(gogoproto.nullable) = false]; + + // Command specifies the SQL commands to which the policy applies. + optional cockroach.sql.catalog.catpb.PolicyCommand command = 4 [(gogoproto.nullable) = false]; } // A DescriptorMutation represents a column or an index that diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 0ff7d7fb9e8a..58a472364fa9 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -2081,6 +2081,14 @@ func (desc *wrapper) validatePolicies() error { p.ID, p.Name, other) } idToName[p.ID] = p.Name + if _, ok := catpb.PolicyType_name[int32(p.Type)]; !ok || p.Type == catpb.PolicyType_POLICYTYPE_UNUSED { + return errors.AssertionFailedf( + "policy %q has an unknown policy type %v", p.Name, p.Type) + } + if _, ok := catpb.PolicyCommand_name[int32(p.Command)]; !ok || p.Command == catpb.PolicyCommand_POLICYCOMMAND_UNUSED { + return errors.AssertionFailedf( + "policy %q has an unknown policy command %v", p.Name, p.Command) + } } return nil } diff --git a/pkg/sql/catalog/tabledesc/validate_test.go b/pkg/sql/catalog/tabledesc/validate_test.go index ac48cdd76d79..dda790034dbb 100644 --- a/pkg/sql/catalog/tabledesc/validate_test.go +++ b/pkg/sql/catalog/tabledesc/validate_test.go @@ -3107,12 +3107,16 @@ func TestValidateTableDesc(t *testing.T) { desc.NextPolicyID = 3 desc.Policies = []descpb.PolicyDescriptor{ { - ID: 1, - Name: "pol", + ID: 1, + Name: "pol", + Type: catpb.PolicyType_PERMISSIVE, + Command: catpb.PolicyCommand_ALL, }, { - ID: 2, - Name: "pol", + ID: 2, + Name: "pol", + Type: catpb.PolicyType_RESTRICTIVE, + Command: catpb.PolicyCommand_INSERT, }, } }), @@ -3122,12 +3126,16 @@ func TestValidateTableDesc(t *testing.T) { desc.NextPolicyID = 11 desc.Policies = []descpb.PolicyDescriptor{ { - ID: 10, - Name: "pol_old", + ID: 10, + Name: "pol_old", + Type: catpb.PolicyType_RESTRICTIVE, + Command: catpb.PolicyCommand_UPDATE, }, { - ID: 10, - Name: "pol_new", + ID: 10, + Name: "pol_new", + Type: catpb.PolicyType_PERMISSIVE, + Command: catpb.PolicyCommand_DELETE, }, } }), @@ -3137,8 +3145,36 @@ func TestValidateTableDesc(t *testing.T) { desc.NextPolicyID = 5 desc.Policies = []descpb.PolicyDescriptor{ { - ID: 20, - Name: "pol", + ID: 20, + Name: "pol", + Type: catpb.PolicyType_PERMISSIVE, + Command: catpb.PolicyCommand_SELECT, + }, + } + }), + }, + {err: `policy "pol" has an unknown policy type POLICYTYPE_UNUSED`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 2 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 1, + Name: "pol", + Type: 0, + Command: catpb.PolicyCommand_ALL, + }, + } + }), + }, + {err: `policy "pol" has an unknown policy command POLICYCOMMAND_UNUSED`, + desc: ModifyDescriptor(func(desc *descpb.TableDescriptor) { + desc.NextPolicyID = 2 + desc.Policies = []descpb.PolicyDescriptor{ + { + ID: 1, + Name: "pol", + Type: catpb.PolicyType_PERMISSIVE, + Command: 0, }, } }), diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_security b/pkg/sql/logictest/testdata/logic_test/row_level_security index 946fb57687b1..545e458a5f54 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_security +++ b/pkg/sql/logictest/testdata/logic_test/row_level_security @@ -107,4 +107,58 @@ DROP TABLE explicit1; statement ok SET use_declarative_schema_changer = $use_decl_sc; +subtest policy_type_and_command_ddl + +statement ok +CREATE TABLE multi_pol_tab1 (c1 INT NOT NULL PRIMARY KEY) + +statement ok +CREATE POLICY "policy 1" ON multi_pol_tab1 AS PERMISSIVE + +statement ok +CREATE POLICY "policy 2" ON multi_pol_tab1 AS RESTRICTIVE + +statement ok +CREATE POLICY "policy 3" ON multi_pol_tab1 FOR ALL + +statement ok +CREATE POLICY "policy 4" ON multi_pol_tab1 FOR INSERT + +statement ok +CREATE POLICY "policy 5" ON multi_pol_tab1 FOR UPDATE + +statement ok +CREATE POLICY "policy 6" ON multi_pol_tab1 FOR DELETE + +statement ok +CREATE POLICY "policy 7" ON multi_pol_tab1 FOR SELECT + +query TT +SHOW CREATE TABLE multi_pol_tab1 +---- +multi_pol_tab1 CREATE TABLE public.multi_pol_tab1 ( + c1 INT8 NOT NULL, + CONSTRAINT multi_pol_tab1_pkey PRIMARY KEY (c1 ASC) + ) + +statement ok +DROP POLICY "policy 1" ON multi_pol_tab1 + +statement ok +DROP POLICY "policy 3" ON multi_pol_tab1 + +statement ok +DROP POLICY "policy 5" ON multi_pol_tab1 + +query TT +SHOW CREATE TABLE multi_pol_tab1 +---- +multi_pol_tab1 CREATE TABLE public.multi_pol_tab1 ( + c1 INT8 NOT NULL, + CONSTRAINT multi_pol_tab1_pkey PRIMARY KEY (c1 ASC) + ) + +statement ok +DROP TABLE multi_pol_tab1 + subtest end diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go index 142136e08644..533f4e64d233 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/create_policy.go @@ -6,11 +6,13 @@ package scbuildstmt import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "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/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/errors" ) // CreatePolicy implements CREATE POLICY. @@ -40,6 +42,8 @@ func CreatePolicy(b BuildCtx, n *tree.CreatePolicy) { b.Add(&scpb.Policy{ TableID: tbl.TableID, PolicyID: policyID, + Type: convertPolicyType(n.Type), + Command: convertPolicyCommand(n.Cmd), }) b.Add(&scpb.PolicyName{ TableID: tbl.TableID, @@ -47,3 +51,33 @@ func CreatePolicy(b BuildCtx, n *tree.CreatePolicy) { Name: string(n.PolicyName), }) } + +// convertPolicyType will convert from a tree.PolicyType to a catpb.PolicyType +func convertPolicyType(in tree.PolicyType) catpb.PolicyType { + switch in { + case tree.PolicyTypeDefault, tree.PolicyTypePermissive: + return catpb.PolicyType_PERMISSIVE + case tree.PolicyTypeRestrictive: + return catpb.PolicyType_RESTRICTIVE + default: + panic(errors.AssertionFailedf("cannot convert tree.PolicyType: %v", in)) + } +} + +// convertPolicyCommand will convert from a tree.PolicyCommand to a catpb.PolicyCommand +func convertPolicyCommand(in tree.PolicyCommand) catpb.PolicyCommand { + switch in { + case tree.PolicyCommandDefault, tree.PolicyCommandAll: + return catpb.PolicyCommand_ALL + case tree.PolicyCommandSelect: + return catpb.PolicyCommand_SELECT + case tree.PolicyCommandInsert: + return catpb.PolicyCommand_INSERT + case tree.PolicyCommandUpdate: + return catpb.PolicyCommand_UPDATE + case tree.PolicyCommandDelete: + return catpb.PolicyCommand_DELETE + default: + panic(errors.AssertionFailedf("cannot convert tree.PolicyCommand: %v", in)) + } +} diff --git a/pkg/sql/schemachanger/scbuild/testdata/create_policy b/pkg/sql/schemachanger/scbuild/testdata/create_policy new file mode 100644 index 000000000000..de2c683ae269 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/testdata/create_policy @@ -0,0 +1,52 @@ +setup +CREATE TABLE defaultdb.foo (i INT PRIMARY KEY); +CREATE USER fred; +---- + +build +CREATE POLICY "first policy" on defaultdb.foo AS PERMISSIVE FOR SELECT TO fred USING (i > 0) WITH CHECK (i % 2 = 0); +---- +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} +- [[Policy:{DescID: 104, PolicyID: 1}, PUBLIC], ABSENT] + {command: 2, policyId: 1, tableId: 104, type: 1} +- [[PolicyName:{DescID: 104, Name: first policy, PolicyID: 1}, PUBLIC], ABSENT] + {name: first policy, policyId: 1, tableId: 104} + +build +CREATE POLICY "second policy" on defaultdb.foo AS RESTRICTIVE FOR INSERT USING (false); +---- +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} +- [[Policy:{DescID: 104, PolicyID: 1}, PUBLIC], ABSENT] + {command: 3, policyId: 1, tableId: 104, type: 2} +- [[PolicyName:{DescID: 104, Name: second policy, PolicyID: 1}, PUBLIC], ABSENT] + {name: second policy, policyId: 1, tableId: 104} + +build +CREATE POLICY "third policy" on defaultdb.foo FOR DELETE TO CURRENT_USER,fred WITH CHECK (i < 0); +---- +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} +- [[Policy:{DescID: 104, PolicyID: 1}, PUBLIC], ABSENT] + {command: 5, policyId: 1, tableId: 104, type: 1} +- [[PolicyName:{DescID: 104, Name: third policy, PolicyID: 1}, PUBLIC], ABSENT] + {name: third policy, policyId: 1, tableId: 104} + +build +CREATE POLICY "fourth policy" on defaultdb.foo AS PERMISSIVE TO PUBLIC,CURRENT_SESSION; +---- +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} +- [[Policy:{DescID: 104, PolicyID: 1}, PUBLIC], ABSENT] + {command: 1, policyId: 1, tableId: 104, type: 1} +- [[PolicyName:{DescID: 104, Name: fourth policy, PolicyID: 1}, PUBLIC], ABSENT] + {name: fourth policy, policyId: 1, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_policy b/pkg/sql/schemachanger/scbuild/testdata/drop_policy new file mode 100644 index 000000000000..812dd312955a --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_policy @@ -0,0 +1,57 @@ +setup +SET enable_row_level_security = true; +CREATE TABLE defaultdb.foo (i INT PRIMARY KEY); +CREATE USER fred; +CREATE POLICY "first policy" on defaultdb.foo AS PERMISSIVE FOR SELECT TO fred USING (i > 0) WITH CHECK (i % 2 = 0); +CREATE POLICY "second policy" on defaultdb.foo AS RESTRICTIVE FOR INSERT USING (false); +CREATE POLICY "third policy" on defaultdb.foo FOR DELETE TO CURRENT_USER,fred WITH CHECK (i < 0); +CREATE POLICY "fourth policy" on defaultdb.foo AS PERMISSIVE TO PUBLIC,CURRENT_SESSION; +---- + +build +DROP POLICY "first policy" on defaultdb.foo; +---- +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[Policy:{DescID: 104, PolicyID: 1}, ABSENT], PUBLIC] + {command: 2, policyId: 1, tableId: 104, type: 1} +- [[PolicyName:{DescID: 104, Name: first policy, PolicyID: 1}, ABSENT], PUBLIC] + {name: first policy, policyId: 1, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} + +build +DROP POLICY "second policy" on defaultdb.foo CASCADE; +---- +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[Policy:{DescID: 104, PolicyID: 2}, ABSENT], PUBLIC] + {command: 3, policyId: 2, tableId: 104, type: 2} +- [[PolicyName:{DescID: 104, Name: second policy, PolicyID: 2}, ABSENT], PUBLIC] + {name: second policy, policyId: 2, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} + +build +DROP POLICY "third policy" on defaultdb.foo RESTRICT; +---- +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[Policy:{DescID: 104, PolicyID: 3}, ABSENT], PUBLIC] + {command: 5, policyId: 3, tableId: 104, type: 1} +- [[PolicyName:{DescID: 104, Name: third policy, PolicyID: 3}, ABSENT], PUBLIC] + {name: third policy, policyId: 3, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} + +build +DROP POLICY "fourth policy" on defaultdb.foo; +---- +- [[IndexData:{DescID: 104, IndexID: 1}, PUBLIC], PUBLIC] + {indexId: 1, tableId: 104} +- [[Policy:{DescID: 104, PolicyID: 4}, ABSENT], PUBLIC] + {command: 1, policyId: 4, tableId: 104, type: 1} +- [[PolicyName:{DescID: 104, Name: fourth policy, PolicyID: 4}, ABSENT], PUBLIC] + {name: fourth policy, policyId: 4, tableId: 104} +- [[TableData:{DescID: 104, ReferencedDescID: 100}, PUBLIC], PUBLIC] + {databaseId: 100, tableId: 104} diff --git a/pkg/sql/schemachanger/scdecomp/decomp.go b/pkg/sql/schemachanger/scdecomp/decomp.go index c3e5f526b653..a9f626e8a7d5 100644 --- a/pkg/sql/schemachanger/scdecomp/decomp.go +++ b/pkg/sql/schemachanger/scdecomp/decomp.go @@ -878,6 +878,8 @@ func (w *walkCtx) walkPolicy(tbl catalog.TableDescriptor, p *descpb.PolicyDescri w.ev(scpb.Status_PUBLIC, &scpb.Policy{ TableID: tbl.GetID(), PolicyID: p.ID, + Type: p.Type, + Command: p.Command, }) w.ev(scpb.Status_PUBLIC, &scpb.PolicyName{ TableID: tbl.GetID(), diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/policy.go b/pkg/sql/schemachanger/scexec/scmutationexec/policy.go index c10e78b13b72..339a78292dc6 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/policy.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/policy.go @@ -22,7 +22,9 @@ func (i *immediateVisitor) AddPolicy(ctx context.Context, op scop.AddPolicy) err tbl.NextPolicyID = op.Policy.PolicyID + 1 } tbl.Policies = append(tbl.Policies, descpb.PolicyDescriptor{ - ID: op.Policy.PolicyID, + ID: op.Policy.PolicyID, + Type: op.Policy.Type, + Command: op.Policy.Command, }) return nil } diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index 10f06ce9efd0..bc41eee2e607 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -368,6 +368,8 @@ message SchemaChild { message Policy { uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; uint32 policy_id = 2 [(gogoproto.customname) = "PolicyID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.PolicyID"]; + uint32 type = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb.PolicyType"]; + uint32 command = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb.PolicyCommand"]; } // PolicyName is the name assigned to a specific policy, based on its ID. diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml index 7b0cb398782c..12ff2a075186 100644 --- a/pkg/sql/schemachanger/scpb/uml/table.puml +++ b/pkg/sql/schemachanger/scpb/uml/table.puml @@ -302,6 +302,8 @@ object Policy Policy : TableID Policy : PolicyID +Policy : Type +Policy : Command object PolicyName diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain index 7fd679bc32e9..68851f9d4fe2 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.explain @@ -11,7 +11,7 @@ Schema change plan for CREATE POLICY ‹"policy 1"› ON ‹t1› AS PERMISSIVE │ │ ├── ABSENT → PUBLIC Policy:{DescID: 104 (t1), PolicyID: 1} │ │ └── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} │ └── 2 Mutation operations - │ ├── AddPolicy {"Policy":{"PolicyID":1,"TableID":104}} + │ ├── AddPolicy {"Policy":{"Command":2,"PolicyID":1,"TableID":104,"Type":1}} │ └── SetPolicyName {"Name":"policy 1","PolicyID":1,"TableID":104} └── PreCommitPhase ├── Stage 1 of 2 in PreCommitPhase @@ -25,5 +25,5 @@ Schema change plan for CREATE POLICY ‹"policy 1"› ON ‹t1› AS PERMISSIVE │ ├── ABSENT → PUBLIC Policy:{DescID: 104 (t1), PolicyID: 1} │ └── ABSENT → PUBLIC PolicyName:{DescID: 104 (t1), Name: "policy 1", PolicyID: 1} └── 2 Mutation operations - ├── AddPolicy {"Policy":{"PolicyID":1,"TableID":104}} + ├── AddPolicy {"Policy":{"Command":2,"PolicyID":1,"TableID":104,"Type":1}} └── SetPolicyName {"Name":"policy 1","PolicyID":1,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects index 7e74501f6fac..38d60dedde21 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_policy/create_policy.side_effects @@ -19,8 +19,10 @@ upsert descriptor #104 + nextPolicyId: 2 parentId: 100 + policies: - + - id: 1 + + - command: SELECT + + id: 1 + name: policy 1 + + type: PERMISSIVE primaryIndex: constraintId: 1 ... @@ -41,8 +43,10 @@ upsert descriptor #104 + nextPolicyId: 2 parentId: 100 + policies: - + - id: 1 + + - command: SELECT + + id: 1 + name: policy 1 + + type: PERMISSIVE primaryIndex: constraintId: 1 ... diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain index 62d9ecdf1557..56886d363680 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.explain @@ -15,7 +15,7 @@ Schema change plan for DROP POLICY ‹"policy 2"› ON ‹t1›; │ │ └── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} │ └── 2 Mutation operations │ ├── SetPolicyName {"Name":"crdb_internal_po...","PolicyID":2,"TableID":104} - │ └── RemovePolicy {"Policy":{"PolicyID":2,"TableID":104}} + │ └── RemovePolicy {"Policy":{"Command":3,"PolicyID":2,"TableID":104,"Type":1}} └── PreCommitPhase ├── Stage 1 of 2 in PreCommitPhase │ ├── 2 elements transitioning toward ABSENT @@ -29,4 +29,4 @@ Schema change plan for DROP POLICY ‹"policy 2"› ON ‹t1›; │ └── PUBLIC → ABSENT PolicyName:{DescID: 104 (t1), Name: "policy 2", PolicyID: 2} └── 2 Mutation operations ├── SetPolicyName {"Name":"crdb_internal_po...","PolicyID":2,"TableID":104} - └── RemovePolicy {"Policy":{"PolicyID":2,"TableID":104}} + └── RemovePolicy {"Policy":{"Command":3,"PolicyID":2,"TableID":104,"Type":1}} diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects index 3b49e01b77ca..a3d5c38e43ad 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_policy/drop_policy.side_effects @@ -17,10 +17,12 @@ increment telemetry for sql.schema.drop_policy ## StatementPhase stage 1 of 1 with 2 MutationType ops upsert descriptor #104 ... - - id: 1 name: policy 1 - - - id: 2 + type: PERMISSIVE + - - command: INSERT + - id: 2 - name: policy 2 + - type: PERMISSIVE primaryIndex: constraintId: 1 ... @@ -36,10 +38,12 @@ persist all catalog changes to storage ## PreCommitPhase stage 2 of 2 with 2 MutationType ops upsert descriptor #104 ... - - id: 1 name: policy 1 - - - id: 2 + type: PERMISSIVE + - - command: INSERT + - id: 2 - name: policy 2 + - type: PERMISSIVE primaryIndex: constraintId: 1 ... diff --git a/pkg/testutils/lint/passes/redactcheck/redactcheck.go b/pkg/testutils/lint/passes/redactcheck/redactcheck.go index e6166aa736a6..e86099e6c49c 100644 --- a/pkg/testutils/lint/passes/redactcheck/redactcheck.go +++ b/pkg/testutils/lint/passes/redactcheck/redactcheck.go @@ -173,8 +173,10 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "LicType": {}, }, "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb": { - "JobID": {}, - "ScheduleID": {}, + "JobID": {}, + "PolicyCommand": {}, + "PolicyType": {}, + "ScheduleID": {}, }, "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb": { "ConstraintValidity": {}, From 60e090960bfacbca68c420e171cf7c79c36b5459 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Mon, 9 Dec 2024 08:09:15 -0500 Subject: [PATCH 071/126] catalog/lease: add a fast way to detect descriptor changes Previously, we had no way to know if any new descriptor versions exist without asking to lease them out again. This could make memo staleness checks for queries expensive, since they would need to re-resolve every single descriptor. To address this, this patch adds the idea of a lease generation counter that is incremented every time a new descriptor version is available. Also to make it easier to detect if new objects exist within a search path, this patch forces object creators to wait for the lease manager to acknowledge the existence of new descriptors (specifically ones that are created in an already resolved schema). Release note: None --- docs/generated/metrics/metrics.html | 1 + pkg/sql/catalog/descpb/lease.proto | 3 + pkg/sql/catalog/descs/collection.go | 7 + pkg/sql/catalog/descs/leased_descriptors.go | 2 + pkg/sql/catalog/lease/count.go | 46 ++- pkg/sql/catalog/lease/lease.go | 380 ++++++++++++++++-- pkg/sql/catalog/lease/lease_test.go | 9 +- pkg/sql/catalog/lease/storage.go | 1 + pkg/sql/catalog/tabledesc/structured.go | 7 + pkg/sql/conn_executor.go | 17 +- pkg/sql/conn_executor_jobs.go | 33 +- .../testdata/logic_test/schema_repair | 5 + pkg/sql/opt/memo/memo.go | 1 - pkg/sql/regions/region_provider_test.go | 4 + pkg/sql/testdata/telemetry/error | 7 + 15 files changed, 465 insertions(+), 58 deletions(-) diff --git a/docs/generated/metrics/metrics.html b/docs/generated/metrics/metrics.html index 24e113d71886..dd32d36b4147 100644 --- a/docs/generated/metrics/metrics.html +++ b/docs/generated/metrics/metrics.html @@ -1720,6 +1720,7 @@ APPLICATIONsql.insights.anomaly_detection.memoryCurrent memory used to support anomaly detectionMemoryGAUGEBYTESAVGNONE APPLICATIONsql.leases.activeThe number of outstanding SQL schema leases.Outstanding leasesGAUGECOUNTAVGNONE APPLICATIONsql.leases.expiredThe number of outstanding session based SQL schema leases expired.Leases expired because of a new versionGAUGECOUNTAVGNONE +APPLICATIONsql.leases.long_wait_for_initial_versionThe number of wait for initial version routines taking more than the lease duration.Number of wait for initial version routines executingGAUGECOUNTAVGNONE APPLICATIONsql.leases.long_wait_for_no_versionThe number of wait for no versions that are taking more than the lease duration.Number of wait for long wait for no version routines executingGAUGECOUNTAVGNONE APPLICATIONsql.leases.long_wait_for_one_versionThe number of wait for one versions that are taking more than the lease duration.Number of wait for long wait for one version routines executingGAUGECOUNTAVGNONE APPLICATIONsql.leases.long_wait_for_two_version_invariantThe number of two version invariant waits that are taking more than the lease duration.Number of two version invariant wait routines executingGAUGECOUNTAVGNONE diff --git a/pkg/sql/catalog/descpb/lease.proto b/pkg/sql/catalog/descpb/lease.proto index aaaf1f8144d1..94beca686f06 100644 --- a/pkg/sql/catalog/descpb/lease.proto +++ b/pkg/sql/catalog/descpb/lease.proto @@ -32,4 +32,7 @@ message WaitStats { (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" ]; + // TargetCount the number of leases we expected to observe, which will be + // non-zero for wait for initial version. + int32 target_count = 7; } diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index 24fe2d5ab15b..6dc95659b800 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -210,6 +210,13 @@ func (tc *Collection) ReleaseAll(ctx context.Context) { tc.skipValidationOnWrite = false } +// GetLeaseGeneration provides an integer which will change whenever new +// descriptor versions are available. This can be used for fast comparisons +// to make sure previously looked up information is still valid. +func (tc *Collection) GetLeaseGeneration() int64 { + return tc.leased.lm.GetLeaseGeneration() +} + // HasUncommittedTables returns true if the Collection contains uncommitted // tables. func (tc *Collection) HasUncommittedTables() (has bool) { diff --git a/pkg/sql/catalog/descs/leased_descriptors.go b/pkg/sql/catalog/descs/leased_descriptors.go index 9d5f10490f8b..57005191c935 100644 --- a/pkg/sql/catalog/descs/leased_descriptors.go +++ b/pkg/sql/catalog/descs/leased_descriptors.go @@ -38,6 +38,8 @@ type LeaseManager interface { ) (decrAfterWait func()) GetSafeReplicationTS() hlc.Timestamp + + GetLeaseGeneration() int64 } type deadlineHolder interface { diff --git a/pkg/sql/catalog/lease/count.go b/pkg/sql/catalog/lease/count.go index 11c6bad3bd1b..31b7e32a2339 100644 --- a/pkg/sql/catalog/lease/count.go +++ b/pkg/sql/catalog/lease/count.go @@ -31,6 +31,8 @@ import ( type countDetail struct { // count is the number of unexpired leases count int + // targetCount is the target number we are trying to reach. + targetCount int // numSQLInstances is the number of distinct SQL instances with unexpired leases. numSQLInstances int // sampleSQLInstanceID is one of the sql_instance_id values we are waiting on, @@ -292,23 +294,7 @@ func countLeasesByRegion( } else { err = queryRegionRows(ctx) } - if err != nil { - if regionliveness.IsQueryTimeoutErr(err) { - // Probe and mark the region potentially. - probeErr := prober.ProbeLiveness(ctx, region) - if probeErr != nil { - err = errors.WithSecondaryError(err, probeErr) - return err - } - return errors.Wrapf(err, "count-lease timed out reading from a region") - } else if regionliveness.IsMissingRegionEnumErr(err) { - // Skip this region because we were unable to find region in - // type descriptor. Since the database regions are cached, they - // may be stale and have dropped regions. - log.Infof(ctx, "count-lease is skipping region %s because of the "+ - "following error %v", region, err) - return nil - } + if err := handleRegionLivenessErrors(ctx, prober, region, err); err != nil { return err } if values == nil { @@ -343,3 +329,29 @@ func getCountLeaseColumns(usesOldSchema bool) string { sb.WriteString(`, count(distinct sql_instance_id), ifnull(min(sql_instance_id),0)`) return sb.String() } + +// handleRegionLivenessErrors handles errors that are linked to region liveness +// timeouts. +func handleRegionLivenessErrors( + ctx context.Context, prober regionliveness.Prober, region string, err error, +) error { + if err != nil { + if regionliveness.IsQueryTimeoutErr(err) { + // Probe and mark the region potentially. + probeErr := prober.ProbeLiveness(ctx, region) + if probeErr != nil { + err = errors.WithSecondaryError(err, probeErr) + return err + } + return errors.Wrapf(err, "count-lease timed out reading from a region") + } else if regionliveness.IsMissingRegionEnumErr(err) { + // Skip this region because we were unable to find region in + // type descriptor. Since the database regions are cached, they + // may be stale and have dropped regions. + log.Infof(ctx, "count-lease skipping region %s due to error: %v", region, err) + return nil + } + return err + } + return err +} diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 6439d19e98b6..d8899700fce6 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -11,6 +11,7 @@ import ( "fmt" "sort" "strconv" + "strings" "sync" "sync/atomic" "time" @@ -30,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/internal/catkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/enum" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/regionliveness" @@ -88,6 +90,11 @@ var LeaseMonitorRangeFeedResetTime = settings.RegisterDurationSetting( time.Minute*25, ) +var WaitForInitialVersion = settings.RegisterBoolSetting(settings.ApplicationLevel, + "sql.catalog.descriptor_wait_for_initial_version.enabled", + "enables waiting for the initial version of a descriptor", + true) + //go:generate stringer -type=SessionBasedLeasingMode type SessionBasedLeasingMode int64 @@ -204,6 +211,264 @@ func (m *Manager) WaitForNoVersion( return nil } +// maybeGetDescriptorWithoutValidation gets a descriptor without validating +// from the KV layer. +func (m *Manager) maybeGetDescriptorWithoutValidation( + ctx context.Context, id descpb.ID, +) (desc catalog.Descriptor, err error) { + err = m.storage.db.KV().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + const isDescriptorRequired = false + cr := m.storage.newCatalogReader(ctx) + c, err := cr.GetByIDs(ctx, txn, []descpb.ID{id}, isDescriptorRequired, catalog.Any) + if err != nil { + return err + } + desc = c.LookupDescriptor(id) + if desc == nil { + // Descriptor was dropped on us, so return a structured error. If + // WaitForInitialVersion is the caller then this indicates the descriptor + // was dropped right after creation. + return errors.Wrapf(catalog.ErrDescriptorNotFound, "descriptor %d could not be fetched to count leases", id) + } + return nil + }) + return desc, err +} + +// countDescriptorsHeldBySessionIDs can be used to make sure certain nodes +// (sessions) observe the existence of a given descriptor. Assuming the given +// sessions are still alive. +func countDescriptorsHeldBySessionIDs( + ctx context.Context, + txn isql.Txn, + descID descpb.ID, + region string, + sessionIDs []sqlliveness.SessionID, +) (int, error) { + regionClause := "" + if region != "" { + regionClause = fmt.Sprintf("AND crdb_region='%s'", region) + } + b := strings.Builder{} + for _, sessionID := range sessionIDs { + if b.Len() > 0 { + b.WriteString(",") + } + b.WriteString(fmt.Sprintf("x'%s'", sessionID.String())) + } + // Query the count from the region. + row, err := txn.QueryRow(ctx, "wait-for-new-descriptor", txn.KV(), + fmt.Sprintf( + ` +SELECT count(*) + FROM system.lease + WHERE desc_id = %d + AND session_id IN (%s) + AND crdb_internal.sql_liveness_is_alive(session_id) + %s;`, + descID, + b.String(), + regionClause, + ), + ) + if err != nil { + return 0, err + } + return int(tree.MustBeDInt(row[0])), nil +} + +// getSessionsHoldingDescriptor can be used to fetch on a per-region basis the +// sessionIDs that are currently holding a lease on descID. If region is empty, +// then all regions will be queried. +func getSessionsHoldingDescriptor( + ctx context.Context, txn isql.Txn, descID descpb.ID, region string, +) ([]sqlliveness.SessionID, error) { + queryStr := ` +SELECT DISTINCT session_id FROM system.lease WHERE desc_id=%d AND crdb_internal.sql_liveness_is_alive(session_id) +` + if region != "" { + queryStr += fmt.Sprintf(" AND crdb_region='%s'", region) + } + rows, err := txn.QueryBuffered(ctx, "active-schema-leases-by-region", txn.KV(), + fmt.Sprintf(queryStr, + descID)) + if err != nil { + return nil, err + } + sessionIDs := make([]sqlliveness.SessionID, 0, len(rows)) + for _, row := range rows { + sessionIDs = append(sessionIDs, sqlliveness.SessionID(tree.MustBeDBytes(row[0]))) + } + return sessionIDs, nil +} + +// WaitForInitialVersion waits for a lease to be acquired on a newly created +// object on any nodes that have already leased the schema out. This ensures +// that their leaseGeneration is incremented before the user commit completes, +// which will ensure that any existing cached queries will detect the new object +// (i.e. the optimizer memo will use the generation value as short circuit). +func (m *Manager) WaitForInitialVersion( + ctx context.Context, + id descpb.ID, + retryOpts retry.Options, + regions regionliveness.CachedDatabaseRegions, +) error { + if !WaitForInitialVersion.Get(&m.settings.SV) { + return nil + } + wsTracker := startWaitStatsTracker(ctx) + defer wsTracker.end() + decrAfterWait := m.IncGaugeAfterLeaseDuration(GaugeWaitForInitialVersion) + defer decrAfterWait() + for lastCount, r := 0, retry.Start(retryOpts); r.Next(); { + desc, err := m.maybeGetDescriptorWithoutValidation(ctx, id) + if err != nil { + if errors.Is(err, catalog.ErrDescriptorNotFound) { + log.Infof(ctx, "descriptor %d was dropped before initial leases was acquired", id) + return nil + } + return err + } + // If the object no longer exists or isn't fully created then skip this + // check, since there is no initial version. We only need to wait for + // initial versions on tables or types, since their resolution is impacted + // by the search_path variable. So, remote lease manager need to be aware + // to invalidate cached metadata (like optimizer memos). + // We don't need to worry about functions because their signature is stored + // inside the schema descriptor, which will cause an implicit bump invalidating + // cached metadata. + if desc == nil || + (desc.DescriptorType() != catalog.Table && desc.DescriptorType() != catalog.Type) || + desc.Dropped() || + desc.Adding() { + return nil + } + // Check to see if there are any leases that still exist on the previous + // version of the descriptor. + now := m.storage.clock.Now() + var count int + db := m.storage.db + // Get a list of sessions that had the schema leased out when this descriptor + // was created / modified. + var sessionsPerRegion map[string][]sqlliveness.SessionID + expectedSessions := 0 + if err := db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + txn.KV().SetDebugName("wait-for-initial-lease-count-schema-leases") + // Look at what was leasing the schema at them modification time, we expect + // this be before the GC TTL because: + // 1) The defaults settings on the system database are not aggressive + // 2) We expect this to be a short wait in the recent past. + // If for some reason this timestamp is outside the GC TTL the transaction + // will get an error, which is a potential caveat here. + if err := txn.KV().SetFixedTimestamp(ctx, desc.GetModificationTime()); err != nil { + return err + } + expectedSessions = 0 + sessionsPerRegion = make(map[string][]sqlliveness.SessionID) + + prober := regionliveness.NewLivenessProber(db.KV(), m.storage.codec, regions, m.settings) + regionMap, err := prober.QueryLiveness(ctx, txn.KV()) + if err != nil { + return err + } + // On single region clusters we can query everything at once. + if regionMap == nil { + sessionIDs, err := getSessionsHoldingDescriptor(ctx, txn, desc.GetParentSchemaID(), "") + if err != nil { + return err + } + sessionsPerRegion[""] = sessionIDs + expectedSessions += len(sessionIDs) + } + // Otherwise, process active schema leases by region, and use the + // region liveness subsystem to detect offline regions. + return regionMap.ForEach(func(region string) error { + var sessionIDs []sqlliveness.SessionID + var err error + if hasTimeout, timeout := prober.GetProbeTimeout(); hasTimeout { + err = timeutil.RunWithTimeout(ctx, "active-schema-leases-by-region", timeout, func(ctx context.Context) error { + var err error + sessionIDs, err = getSessionsHoldingDescriptor(ctx, txn, desc.GetParentSchemaID(), region) + return err + }) + } else { + sessionIDs, err = getSessionsHoldingDescriptor(ctx, txn, desc.GetParentSchemaID(), region) + } + if err != nil { + return handleRegionLivenessErrors(ctx, prober, region, err) + } + sessionsPerRegion[region] = sessionIDs + expectedSessions += len(sessionIDs) + return nil + }) + }); err != nil { + return err + } + // Next ensure the initial version exists on all nodes that have the schema + // leased out. + if err := db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + count = 0 + txn.KV().SetDebugName("wait-for-initial-lease") + if err := txn.KV().SetFixedTimestamp(ctx, now); err != nil { + return err + } + prober := regionliveness.NewLivenessProber(db.KV(), m.storage.codec, regions, m.settings) + regionMap, err := prober.QueryLiveness(ctx, txn.KV()) + if err != nil { + return err + } + // On multi-region we are going to process initial version on a per-region + // basis, which will allow us to query / inform region liveness of offline + // regions. + if regions != nil && regions.IsMultiRegion() { + return regionMap.ForEach(func(region string) error { + sessionIDs := sessionsPerRegion[region] + // Skip any regions without sessions. + if len(sessionIDs) == 0 { + return nil + } + + var regionCount int + var err error + if hasTimeout, timeout := prober.GetProbeTimeout(); hasTimeout { + err = timeutil.RunWithTimeout(ctx, "wait-for-new-descriptor-by-region", timeout, func(ctx context.Context) error { + var err error + regionCount, err = countDescriptorsHeldBySessionIDs(ctx, txn, desc.GetID(), region, sessionIDs) + return err + }) + } else { + regionCount, err = countDescriptorsHeldBySessionIDs(ctx, txn, desc.GetID(), region, sessionIDs) + } + if err := handleRegionLivenessErrors(ctx, prober, region, err); err != nil { + return err + } + count += regionCount + return nil + }) + } else { + // Otherwise, we can query the initial versions directly. + count, err = countDescriptorsHeldBySessionIDs(ctx, txn, desc.GetID(), "", sessionsPerRegion[""]) + return err + } + }); err != nil { + return err + } + if count == expectedSessions { + break + } + if count != lastCount { + log.Infof(ctx, "waiting for descriptor %d to appear on %d nodes. Last count was %d", desc.GetID(), expectedSessions, lastCount) + wsTracker.updateProgress(countDetail{ + count: count, + targetCount: expectedSessions, + }) + } + lastCount = count + + } + return nil +} + // WaitForOneVersion returns once there are no unexpired leases on the // previous version of the descriptor. It returns the descriptor with the // current version, though note that it will not be validated or hydrated. @@ -228,29 +493,10 @@ func (m *Manager) WaitForOneVersion( wsTracker := startWaitStatsTracker(ctx) defer wsTracker.end() for lastCount, r := 0, retry.Start(retryOpts); r.Next(); { - if err := m.storage.db.KV().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - // Use the lower-level MaybeGetDescriptorByIDUnvalidated to avoid - // performing validation while waiting for leases to drain. - // Validation is somewhat expensive but more importantly, is not - // particularly desirable in this context: there are valid cases where - // descriptors can be removed or made invalid. For instance, the - // descriptor could be a type or a schema which is dropped by a subsequent - // concurrent schema change. - const isDescriptorRequired = false - cr := m.storage.newCatalogReader(ctx) - c, err := cr.GetByIDs(ctx, txn, []descpb.ID{id}, isDescriptorRequired, catalog.Any) - if err != nil { - return err - } - desc = c.LookupDescriptor(id) - if desc == nil { - return errors.Wrapf(catalog.ErrDescriptorNotFound, "waiting for leases to drain on descriptor %d", id) - } - return nil - }); err != nil { + var err error + if desc, err = m.maybeGetDescriptorWithoutValidation(ctx, id); err != nil { return nil, err } - // Check to see if there are any leases that still exist on the previous // version of the descriptor. now := m.storage.clock.Now() @@ -429,10 +675,18 @@ func getDescriptorsFromStoreForInterval( if err != nil { return err } + // For offline tables the modification time is never set by the builder, + // which can break historical queries. So, manually set this value here. + desc := descBuilder.BuildImmutable() + if desc.Offline() && desc.DescriptorType() == catalog.Table { + descMut := descBuilder.BuildExistingMutable() + descMut.(*tabledesc.Mutable).ForceModificationTime(k.Timestamp) + desc = descMut + } // Construct a historical descriptor with expiration. histDesc := historicalDescriptor{ - desc: descBuilder.BuildImmutable(), + desc: desc, expiration: subsequentModificationTime, } descriptorsRead = append(descriptorsRead, histDesc) @@ -894,6 +1148,10 @@ type Manager struct { descDelCh chan descpb.ID // rangefeedErrCh receives any terminal errors from the rangefeed. rangefeedErrCh chan error + // leaseGeneration increments any time a new or existing descriptor is + // detected by the lease manager. Once this count is incremented new data + // is available. + leaseGeneration atomic.Int64 } const leaseConcurrencyLimit = 5 @@ -965,6 +1223,12 @@ func NewLeaseManager( Measurement: "Number of two version invariant wait routines executing", Unit: metric.Unit_COUNT, }), + longWaitForInitialVersionActive: metric.NewGauge(metric.Metadata{ + Name: "sql.leases.long_wait_for_initial_version", + Help: "The number of wait for initial version routines taking more than the lease duration.", + Measurement: "Number of wait for initial version routines executing", + Unit: metric.Unit_COUNT, + }), }, }, settings: settings, @@ -1325,6 +1589,18 @@ func (m *Manager) SetDraining( } } +// isDescriptorStateEmpty determines if a descriptor state exists and +// has any active versions inside it. +func (m *Manager) isDescriptorStateEmpty(id descpb.ID) bool { + st := m.findDescriptorState(id, false /* create */) + if st == nil { + return true + } + st.mu.Lock() + defer st.mu.Unlock() + return len(st.mu.active.data) == 0 +} + // If create is set, cache and stopper need to be set as well. func (m *Manager) findDescriptorState(id descpb.ID, create bool) *descriptorState { m.mu.Lock() @@ -1351,7 +1627,9 @@ func (m *Manager) RefreshLeases(ctx context.Context, s *stop.Stopper, db *kv.DB) case id := <-m.descDelCh: // Descriptor is marked as deleted, so mark it for deletion or // remove it if it's no longer in use. - _ = s.RunAsyncTask(ctx, "purge deleted descriptor", func(ctx context.Context) { + _ = s.RunAsyncTask(ctx, "purgeOldVersionsOrAcquireInitialVersion deleted descriptor", func(ctx context.Context) { + // Once the descriptor is purged notify that some change has occurred. + defer m.leaseGeneration.Add(1) state := m.findNewest(id) if state != nil { if err := purgeOldVersions(ctx, db, id, true /* dropped */, state.GetVersion(), m); err != nil { @@ -1390,10 +1668,35 @@ func (m *Manager) RefreshLeases(ctx context.Context, s *stop.Stopper, db *kv.DB) // Try to refresh the lease to one >= this version. log.VEventf(ctx, 2, "purging old version of descriptor %d@%d (dropped %v)", desc.GetID(), desc.GetVersion(), dropped) - purge := func(ctx context.Context) { - if err := purgeOldVersions(ctx, db, desc.GetID(), dropped, desc.GetVersion(), m); err != nil { - log.Warningf(ctx, "error purging leases for descriptor %d(%s): %s", - desc.GetID(), desc.GetName(), err) + // purgeOldVersionsOrAcquireInitialVersion will purge older versions of + // descriptors. Or if a new descriptor shows up then the initial version + // will be acquired if the schema is already leased to invalidate metadata + // caches (like optimizer memos). + purgeOldVersionsOrAcquireInitialVersion := func(ctx context.Context) { + // Notify of any new / modified descriptors below once a new lease is + // acquired. + defer m.leaseGeneration.Add(1) + + // Whenever a new relation / type is created under an already leased + // schema we are going to lease the object out immediately. This allows + // us to use the generation count to cache information like Memo's in + // the optimizer. The creating object will wait for us to acquire the + // lease and acknowledge the existence before that txn can return to + // the user. + if WaitForInitialVersion.Get(&m.settings.SV) && + (!desc.Adding() && !desc.Dropped() && !desc.Offline()) && + desc.GetParentSchemaID() != descpb.InvalidID && + (m.isDescriptorStateEmpty(desc.GetID())) && + m.findDescriptorState(desc.GetParentSchemaID(), false) != nil { + err := ensureVersion(ctx, desc.GetID(), desc.GetVersion(), m) + if err != nil { + log.Warningf(ctx, "error fetching lease for descriptor %s", err) + } + } else { + if err := purgeOldVersions(ctx, db, desc.GetID(), dropped, desc.GetVersion(), m); err != nil { + log.Warningf(ctx, "error purging leases for descriptor %d(%s): %s", + desc.GetID(), desc.GetName(), err) + } } } // New descriptors may appear in the future if the descriptor table is @@ -1407,17 +1710,17 @@ func (m *Manager) RefreshLeases(ctx context.Context, s *stop.Stopper, db *kv.DB) // acquiring the lease on the new version. This could lead to periods // of increased latency right as the descriptor has been committed. if now := db.Clock().Now(); now.Less(desc.GetModificationTime()) { - _ = s.RunAsyncTask(ctx, "wait to purge", func(ctx context.Context) { + _ = s.RunAsyncTask(ctx, "wait to purgeOldVersionsOrAcquireInitialVersion", func(ctx context.Context) { toWait := time.Duration(desc.GetModificationTime().WallTime - now.WallTime) select { case <-time.After(toWait): - purge(ctx) + purgeOldVersionsOrAcquireInitialVersion(ctx) case <-ctx.Done(): case <-s.ShouldQuiesce(): } }) } else { - purge(ctx) + purgeOldVersionsOrAcquireInitialVersion(ctx) } if evFunc := m.testingKnobs.TestingDescriptorRefreshedEvent; evFunc != nil { @@ -1430,6 +1733,13 @@ func (m *Manager) RefreshLeases(ctx context.Context, s *stop.Stopper, db *kv.DB) }) } +// GetLeaseGeneration provides an integer which will change whenever new +// descriptor versions are available. This can be used for fast comparisons +// to make sure previously looked up information is still valid. +func (m *Manager) GetLeaseGeneration() int64 { + return m.leaseGeneration.Load() +} + // GetSafeReplicationTS gets the timestamp till which the leased descriptors // have been synced. func (m *Manager) GetSafeReplicationTS() hlc.Timestamp { @@ -1816,6 +2126,9 @@ func (m *Manager) refreshSomeLeases(ctx context.Context, includeAll bool) { } } wg.Wait() + // Indicate some descriptor has changed at the end of the manual + // refresh. + m.leaseGeneration.Add(1) } // DeleteOrphanedLeases releases all orphaned leases created by a prior @@ -1935,6 +2248,7 @@ type Metrics struct { LongWaitForOneVersionsActive *metric.Gauge LongWaitForNoVersionsActive *metric.Gauge LongWaitForTwoVersionInvariantActive *metric.Gauge + LongWaitForInitialVersionActive *metric.Gauge } // MetricsStruct returns a struct containing all of this Manager's metrics. @@ -1946,6 +2260,7 @@ func (m *Manager) MetricsStruct() Metrics { LongWaitForNoVersionsActive: m.storage.longWaitForNoVersionsActive, LongWaitForOneVersionsActive: m.storage.longWaitForOneVersionsActive, LongWaitForTwoVersionInvariantActive: m.storage.longTwoVersionInvariantViolationWaitActive, + LongWaitForInitialVersionActive: m.storage.longWaitForInitialVersionActive, } } @@ -1999,6 +2314,8 @@ const ( GaugeWaitForNoVersion // GaugeWaitForTwoVersionViolation gauge for CheckTwoVersionInvariant. GaugeWaitForTwoVersionViolation + // GaugeWaitForInitialVersion gauge for WaitForInitialVersion. + GaugeWaitForInitialVersion ) // IncGaugeAfterLeaseDuration increments a wait metric after the lease duration @@ -2014,6 +2331,8 @@ func (m *Manager) IncGaugeAfterLeaseDuration( gauge = m.storage.longWaitForNoVersionsActive case GaugeWaitForTwoVersionViolation: gauge = m.storage.longTwoVersionInvariantViolationWaitActive + case GaugeWaitForInitialVersion: + gauge = m.storage.longWaitForInitialVersionActive default: panic(errors.Newf("unknown gauge type %d", gaugeType)) } @@ -2060,6 +2379,7 @@ func (w *waitStatsTracker) updateProgress(detail countDetail) { w.ws.LastCount = int32(detail.count) w.ws.SampleSqlInstanceId = int32(detail.sampleSQLInstanceID) w.ws.NumSqlInstances = int32(detail.numSQLInstances) + w.ws.TargetCount = int32(detail.targetCount) w.ws.ElapsedTimeInMs = timeutil.Since(w.startTime).Milliseconds() w.recSpan.RecordStructured(&w.ws) } diff --git a/pkg/sql/catalog/lease/lease_test.go b/pkg/sql/catalog/lease/lease_test.go index aed3fb0057fc..141522f2e692 100644 --- a/pkg/sql/catalog/lease/lease_test.go +++ b/pkg/sql/catalog/lease/lease_test.go @@ -2454,6 +2454,10 @@ func TestRangefeedUpdatesHandledProperlyInTheFaceOfRaces(t *testing.T) { db1 := tc.ServerConn(0) tdb1 := sqlutils.MakeSQLRunner(db1) + // Disable acquisition / waiting for the initial version, since the callbacks + // above will get held up on the CREATE. This test needs the SELECT after to + // be the first acquisition. + tdb1.Exec(t, "SET CLUSTER SETTING sql.catalog.descriptor_wait_for_initial_version.enabled=false") db2 := tc.ServerConn(1) // Create a couple of descriptors. @@ -2613,10 +2617,11 @@ func TestLeaseWithOfflineTables(t *testing.T) { checkLeaseState(true /* shouldBePresent */) // Take the table dropped and back online again. - // This should relinquish the lease. + // This should relinquish the lease. Note: We acquire + // leases for new descriptors automatically now. setTableState(descpb.DescriptorState_PUBLIC, descpb.DescriptorState_DROP) setTableState(descpb.DescriptorState_DROP, descpb.DescriptorState_PUBLIC) - checkLeaseState(false /* shouldBePresent */) + checkLeaseState(true /* shouldBePresent */) // Query the table, thereby acquiring a lease once again. runner.CheckQueryResults(t, "SELECT s FROM t.test", [][]string{}) diff --git a/pkg/sql/catalog/lease/storage.go b/pkg/sql/catalog/lease/storage.go index 495d1c9ff7ec..86ca14117c7c 100644 --- a/pkg/sql/catalog/lease/storage.go +++ b/pkg/sql/catalog/lease/storage.go @@ -72,6 +72,7 @@ type leasingMetrics struct { longWaitForOneVersionsActive *metric.Gauge longWaitForNoVersionsActive *metric.Gauge longTwoVersionInvariantViolationWaitActive *metric.Gauge + longWaitForInitialVersionActive *metric.Gauge } type leaseFields struct { diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 0cc9606190d8..0e144ce6642d 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -2774,3 +2774,10 @@ func (desc *Mutable) BumpExternalAsOf(timestamp hlc.Timestamp) error { desc.External.AsOf = timestamp return nil } + +// ForceModificationTime allows the modification time to be set externally. +// Note: This API is only used by the leasing code to adjust modification time +// since the builder will not populate it for offline descriptors. +func (desc *Mutable) ForceModificationTime(modificationTime hlc.Timestamp) { + desc.ModificationTime = modificationTime +} diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index f3dbbf4bb5fb..fb9ea7539e2f 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -52,6 +52,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirecancel" + "github.com/cockroachdb/cockroach/pkg/sql/regions" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" "github.com/cockroachdb/cockroach/pkg/sql/sem/asof" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" @@ -4071,8 +4072,20 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper( handleErr(err) } ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionEndPostCommitJob, crtime.NowMono()) - if err := ex.waitOneVersionForNewVersionDescriptorsWithoutJobs(descIDsInJobs); err != nil { - return advanceInfo{}, err + // If descriptors are either modified or created wait then we may have to + // wait for one version (if no job exists) or the initial version to be + // acquired. + if ex.extraTxnState.descCollection.HasUncommittedDescriptors() { + cachedRegions, err := regions.NewCachedDatabaseRegions(ex.Ctx(), ex.server.cfg.DB, ex.server.cfg.LeaseManager) + if err != nil { + return advanceInfo{}, err + } + if err := ex.waitOneVersionForNewVersionDescriptorsWithoutJobs(descIDsInJobs, cachedRegions); err != nil { + return advanceInfo{}, err + } + if err := ex.waitForInitialVersionForNewDescriptors(cachedRegions); err != nil { + return advanceInfo{}, err + } } if ex.extraTxnState.descCollection.HasUncommittedNewOrDroppedDescriptors() { execCfg := ex.planner.ExecCfg() diff --git a/pkg/sql/conn_executor_jobs.go b/pkg/sql/conn_executor_jobs.go index f4d6c26bcecf..750042ba4e08 100644 --- a/pkg/sql/conn_executor_jobs.go +++ b/pkg/sql/conn_executor_jobs.go @@ -6,14 +6,17 @@ package sql import ( + "time" + "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/regions" + "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/errors" ) -// waitOneVersionForNewVersionDescriptorsWithoutJobs is to used wait until all +// waitOneVersionForNewVersionDescriptorsWithoutJobs is used to wait until all // descriptors with new versions to converge to one version in the cluster. // `descIDsInJobs` are collected with `descIDsInSchemaChangeJobs`. We need to do // this to make sure all descriptors mutated are at one version when the schema @@ -21,7 +24,7 @@ import ( // thing for affected descriptors. But, in some scenario, jobs are not created // for mutated descriptors. func (ex *connExecutor) waitOneVersionForNewVersionDescriptorsWithoutJobs( - descIDsInJobs catalog.DescriptorIDSet, + descIDsInJobs catalog.DescriptorIDSet, cachedRegions *regions.CachedDatabaseRegions, ) error { withNewVersion, err := ex.extraTxnState.descCollection.GetOriginalPreviousIDVersionsForUncommitted() if err != nil { @@ -31,10 +34,6 @@ func (ex *connExecutor) waitOneVersionForNewVersionDescriptorsWithoutJobs( if len(withNewVersion) == 0 { return nil } - cachedRegions, err := regions.NewCachedDatabaseRegions(ex.Ctx(), ex.server.cfg.DB, ex.server.cfg.LeaseManager) - if err != nil { - return err - } for _, idVersion := range withNewVersion { if descIDsInJobs.Contains(idVersion.ID) { continue @@ -55,6 +54,28 @@ func (ex *connExecutor) waitOneVersionForNewVersionDescriptorsWithoutJobs( return nil } +func (ex *connExecutor) waitForInitialVersionForNewDescriptors( + cachedRegions *regions.CachedDatabaseRegions, +) error { + // Detect any tables that have just been created, we will confirm that all + // nodes that have leased the schema for them out are aware of the new object. + // This guarantees that any cached optimizer memos are discarded once the + // user transaction completes. + for _, tbl := range ex.extraTxnState.descCollection.GetUncommittedTables() { + if tbl.GetVersion() == 1 { + err := ex.planner.LeaseMgr().WaitForInitialVersion(ex.Ctx(), tbl.GetID(), retry.Options{ + InitialBackoff: time.Millisecond, + MaxBackoff: time.Second, + Multiplier: 1.5, + }, cachedRegions) + if err != nil { + return err + } + } + } + return nil +} + // descIDsInSchemaChangeJobs returns all descriptor IDs with which schema change // jobs in this transaction will perform. Within schema change jobs, we also // wait until the whole cluster only has leases on the latest version of these diff --git a/pkg/sql/logictest/testdata/logic_test/schema_repair b/pkg/sql/logictest/testdata/logic_test/schema_repair index 631379024b5c..e851ee78a234 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema_repair +++ b/pkg/sql/logictest/testdata/logic_test/schema_repair @@ -1,5 +1,10 @@ subtest lost_table_data +# This test will intentionally corrupt descriptors, so the initial version +# can *never* be acquired. +statement ok +SET CLUSTER SETTING sql.catalog.descriptor_wait_for_initial_version.enabled=false + statement ok CREATE TABLE corruptdesc (v INT8) diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index b2a59254e361..2fbb2c3f6d79 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -466,7 +466,6 @@ func (m *Memo) IsStale( m.txnIsoLevel != evalCtx.TxnIsoLevel { return true, nil } - // Memo is stale if the fingerprint of any object in the memo's metadata has // changed, or if the current user no longer has sufficient privilege to // access the object. diff --git a/pkg/sql/regions/region_provider_test.go b/pkg/sql/regions/region_provider_test.go index da76caacf0a7..7a8192a4b5f7 100644 --- a/pkg/sql/regions/region_provider_test.go +++ b/pkg/sql/regions/region_provider_test.go @@ -214,6 +214,10 @@ func (f fakeLeaseManager) GetSafeReplicationTS() hlc.Timestamp { return hlc.Timestamp{} } +func (f fakeLeaseManager) GetLeaseGeneration() int64 { + return 0 +} + var _ descs.LeaseManager = (*fakeLeaseManager)(nil) type fakeSystemDatabase struct { diff --git a/pkg/sql/testdata/telemetry/error b/pkg/sql/testdata/telemetry/error index 4ce58e39310b..d9655947a6ad 100644 --- a/pkg/sql/testdata/telemetry/error +++ b/pkg/sql/testdata/telemetry/error @@ -12,6 +12,13 @@ rewrite ${1} (...) ---- +# We are injecting a corrupt descriptors below, so lease on first creation +# will not work. +exec +SET CLUSTER SETTING sql.catalog.descriptor_wait_for_initial_version.enabled=false +---- + + exec CREATE TABLE fktbl (id INT PRIMARY KEY); ---- From 61318129f66cc087a4f4089dd0076a9c52c56a1f Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Mon, 9 Dec 2024 12:06:22 -0500 Subject: [PATCH 072/126] sql/stats: add a fast way to detect statistics changes Previously, the only way to know if any of the statistics changed was to query the statistics cache for a specific table. To make memo staleness checks cheaper, this patch adds a generation count to the statistics cache, which will be modified on any statistics update. Release note: None --- pkg/sql/stats/stats_cache.go | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/pkg/sql/stats/stats_cache.go b/pkg/sql/stats/stats_cache.go index 7893477e18fc..cc46d7b483b0 100644 --- a/pkg/sql/stats/stats_cache.go +++ b/pkg/sql/stats/stats_cache.go @@ -11,6 +11,7 @@ import ( "math" "sort" "sync" + "sync/atomic" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -71,6 +72,10 @@ type TableStatisticsCache struct { // Used when decoding KV from the range feed. datumAlloc tree.DatumAlloc + + // generation is incremented any time the statistics cache is + // modified. + generation atomic.Int64 } // The cache stores *cacheEntry objects. The fields are protected by the @@ -135,6 +140,13 @@ func (sc *TableStatisticsCache) Clear() { sc.mu.Lock() defer sc.mu.Unlock() sc.mu.cache.Clear() + defer sc.generation.Add(1) +} + +// GetGeneration returns the current generation, which will change if any +// modifications happen to the cache. +func (sc *TableStatisticsCache) GetGeneration() int64 { + return sc.generation.Load() } // Start begins watching for updates in the stats table. @@ -325,6 +337,7 @@ func (sc *TableStatisticsCache) getTableStatsFromCache( ) ([]*TableStatistic, error) { sc.mu.Lock() defer sc.mu.Unlock() + defer sc.generation.Add(1) if found, e := sc.lookupStatsLocked(ctx, tableID, false /* stealthy */); found { if e.isStale(forecast, udtCols) { @@ -413,6 +426,7 @@ func (sc *TableStatisticsCache) lookupStatsLocked( func (sc *TableStatisticsCache) addCacheEntryLocked( ctx context.Context, tableID descpb.ID, forecast bool, typeResolver *descs.DistSQLTypeResolver, ) (stats []*TableStatistic, err error) { + defer sc.generation.Add(1) // Add a cache entry that other queries can find and wait on until we have the // stats. e := &cacheEntry{ @@ -426,7 +440,6 @@ func (sc *TableStatisticsCache) addCacheEntryLocked( func() { sc.mu.Unlock() defer sc.mu.Lock() - log.VEventf(ctx, 1, "reading statistics for table %d", tableID) stats, udts, err = sc.getTableStatsFromDB(ctx, tableID, forecast, sc.settings, typeResolver) log.VEventf(ctx, 1, "finished reading statistics for table %d", tableID) @@ -459,6 +472,7 @@ func (sc *TableStatisticsCache) refreshCacheEntry( ) { sc.mu.Lock() defer sc.mu.Unlock() + defer sc.generation.Add(1) // If the stats don't already exist in the cache, don't bother performing // the refresh. If e.err is not nil, the stats are in the process of being @@ -532,6 +546,7 @@ func (sc *TableStatisticsCache) InvalidateTableStats(ctx context.Context, tableI sc.mu.Lock() defer sc.mu.Unlock() sc.mu.cache.Del(tableID) + defer sc.generation.Add(1) } const ( From 0275f1f52ebbe91e5dd7b6fbe4176977c0972283 Mon Sep 17 00:00:00 2001 From: Kyle Wong <37189875+kyle-a-wong@users.noreply.github.com> Date: Mon, 13 Jan 2025 15:43:16 -0500 Subject: [PATCH 073/126] roachtest: fix WaitForReady bugs WaitForReady was not properly handling when the deadline of a timeout context was reached while waiting for the cluster to be ready. Previously, hitting the timeout would result in the roachtest running for 10 minutes until the whole test timed out. Now, WaitForReady respects when the timeutil.RunWithTimeout context times out. Updated WaitForReady's checkReady func to use RoachtestHTTPClient's http client when making the call to GET /health?ready=1 instead of RoachtestHTTPClient's wrapper method. This wrapper method attempts to authenticate the user if no session id exists, but this endpoint doesn't require authentication so it doesnt need to use the wrapper method. Fixes: #138143 Resolves: #136128 Epic: None Release note: None --- pkg/cmd/roachtest/roachtestutil/utils.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/cmd/roachtest/roachtestutil/utils.go b/pkg/cmd/roachtest/roachtestutil/utils.go index 7da07e9724d4..40aa4f18024a 100644 --- a/pkg/cmd/roachtest/roachtestutil/utils.go +++ b/pkg/cmd/roachtest/roachtestutil/utils.go @@ -189,7 +189,7 @@ func WaitForReady( ) { client := DefaultHTTPClient(c, t.L()) checkReady := func(ctx context.Context, url string) error { - resp, err := client.Get(ctx, url) + resp, err := client.client.Get(ctx, url) if err != nil { return err } @@ -212,13 +212,13 @@ func WaitForReady( for i, adminAddr := range adminAddrs { url := fmt.Sprintf(`https://%s/health?ready=1`, adminAddr) - for err := checkReady(ctx, url); err != nil; err = checkReady(ctx, url) { + for err := checkReady(ctx, url); err != nil && ctx.Err() == nil; err = checkReady(ctx, url) { t.L().Printf("n%d not ready, retrying: %s", nodes[i], err) time.Sleep(time.Second) } t.L().Printf("n%d is ready", nodes[i]) } - return nil + return ctx.Err() }, )) } From 4365f0e84334af508dbb594177e657b1ee6d9e31 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Tue, 10 Dec 2024 13:18:43 -0500 Subject: [PATCH 074/126] sql: add and adopt fast memo staleness checks Previously, determining if a memo was stale involved re-resolving descriptor references to ensure that the versions and descriptor IDs were the same. This could be fairly expensive and hit various points of contention. To address this, this patch takes advantage of the lease manager and stats cache generation to determine if descriptors have to be re-resolved or new table statistics are available. It does so by introducing a dependency digest that will contain this lease / stats generation, current database, system zone config, current user and search_path. If none of these things change, then we know that a memo can be re-used without any extra checks. This is only adopted for prepared statements currently. Release note: None Fixes: #105867 --- .../testdata/benchmark_expectations | 10 +- pkg/sql/opt/BUILD.bazel | 1 + pkg/sql/opt/cat/BUILD.bazel | 1 + pkg/sql/opt/cat/catalog.go | 46 ++++++ pkg/sql/opt/memo/expr_format.go | 1 + pkg/sql/opt/memo/memo.go | 8 +- pkg/sql/opt/metadata.go | 29 ++++ pkg/sql/opt/testutils/testcat/test_catalog.go | 16 +- pkg/sql/opt_catalog.go | 18 +++ pkg/sql/sessiondata/search_path.go | 2 +- pkg/sql/tests/BUILD.bazel | 1 + pkg/sql/tests/dependency_digest_test.go | 147 ++++++++++++++++++ 12 files changed, 267 insertions(+), 13 deletions(-) create mode 100644 pkg/sql/tests/dependency_digest_test.go diff --git a/pkg/bench/rttanalysis/testdata/benchmark_expectations b/pkg/bench/rttanalysis/testdata/benchmark_expectations index b910e85dba0b..f29c3155c978 100644 --- a/pkg/bench/rttanalysis/testdata/benchmark_expectations +++ b/pkg/bench/rttanalysis/testdata/benchmark_expectations @@ -27,7 +27,7 @@ exp,benchmark 8,AlterTableUnsplit/alter_table_unsplit_at_1_value 11,AlterTableUnsplit/alter_table_unsplit_at_2_values 14,AlterTableUnsplit/alter_table_unsplit_at_3_values -5,Audit/select_from_an_audit_table +2-4,Audit/select_from_an_audit_table 26,CreateRole/create_role_with_1_option 29,CreateRole/create_role_with_2_options 32,CreateRole/create_role_with_3_options @@ -52,7 +52,7 @@ exp,benchmark 17,DropView/drop_2_views 17,DropView/drop_3_views 5,GenerateObjects/generate_1000_tables_-_this_test_should_use_the_same_number_of_RTTs_as_for_10_tables -11,GenerateObjects/generate_100_tables_from_template +8-10,GenerateObjects/generate_100_tables_from_template 5,GenerateObjects/generate_10_tables 16,GenerateObjects/generate_10x10_schemas_and_tables_in_existing_db 5,GenerateObjects/generate_50000_tables @@ -125,9 +125,9 @@ exp,benchmark 20,Truncate/truncate_2_column_0_rows 20,Truncate/truncate_2_column_1_rows 20,Truncate/truncate_2_column_2_rows -3,UDFResolution/select_from_udf -5,UseManyRoles/use_2_roles -3,UseManyRoles/use_50_roles +0,UDFResolution/select_from_udf +2,UseManyRoles/use_2_roles +0,UseManyRoles/use_50_roles 1,VirtualTableQueries/select_crdb_internal.invalid_objects_with_1_fk 1,VirtualTableQueries/select_crdb_internal.tables_with_1_fk 5,VirtualTableQueries/virtual_table_cache_with_point_lookups diff --git a/pkg/sql/opt/BUILD.bazel b/pkg/sql/opt/BUILD.bazel index f299a667ad13..0cf6ff5f10da 100644 --- a/pkg/sql/opt/BUILD.bazel +++ b/pkg/sql/opt/BUILD.bazel @@ -46,6 +46,7 @@ go_library( "//pkg/util/buildutil", "//pkg/util/intsets", "//pkg/util/log", + "//pkg/util/syncutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", "@com_github_lib_pq//oid", diff --git a/pkg/sql/opt/cat/BUILD.bazel b/pkg/sql/opt/cat/BUILD.bazel index eac762111570..4cd147613bc2 100644 --- a/pkg/sql/opt/cat/BUILD.bazel +++ b/pkg/sql/opt/cat/BUILD.bazel @@ -20,6 +20,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/opt/cat", visibility = ["//visibility:public"], deps = [ + "//pkg/config", "//pkg/config/zonepb", "//pkg/geo/geopb", "//pkg/roachpb", diff --git a/pkg/sql/opt/cat/catalog.go b/pkg/sql/opt/cat/catalog.go index b313e63d71bf..b47261aea951 100644 --- a/pkg/sql/opt/cat/catalog.go +++ b/pkg/sql/opt/cat/catalog.go @@ -10,11 +10,13 @@ package cat import ( "context" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/lib/pq/oid" ) @@ -77,6 +79,45 @@ type Flags struct { IncludeNonActiveIndexes bool } +// DependencyDigest can be stored and confirm that all catalog objects resolved, +// at the current point in time will have the same version, stats, valid privileges, +// and name resolution. +type DependencyDigest struct { + // LeaseGeneration tracks if any new version of descriptors has been published + // by the lease manager. + LeaseGeneration int64 + + // StatsGeneration tracks if any new statistics have been published. + StatsGeneration int64 + + // SystemConfig tracks the current system config, which is refreshed on + // any zone config update. + SystemConfig *config.SystemConfig + + CurrentDatabase string + SearchPath sessiondata.SearchPath + CurrentUser username.SQLUsername +} + +// Equal compares if two dependency digests match. +func (d *DependencyDigest) Equal(other *DependencyDigest) bool { + return d.LeaseGeneration == other.LeaseGeneration && + d.StatsGeneration == other.StatsGeneration && + // Note: If the system config is modified a new SystemConfig structure + // is always allocated. Individual fields cannot change on the caller, + // so for the purpose of the dependency digest its sufficient to just + // compare / store pointers. + d.SystemConfig == other.SystemConfig && + d.CurrentDatabase == other.CurrentDatabase && + d.SearchPath.Equals(&other.SearchPath) && + d.CurrentUser == other.CurrentUser +} + +// Clear clears a dependency digest. +func (d *DependencyDigest) Clear() { + *d = DependencyDigest{} +} + // Catalog is an interface to a database catalog, exposing only the information // needed by the query optimizer. // @@ -210,6 +251,11 @@ type Catalog interface { // GetCurrentUser returns the username.SQLUsername of the current session. GetCurrentUser() username.SQLUsername + // GetDependencyDigest fetches the current dependency digest, which can be + // used as a fast comparison to guarantee that all dependencies will resolve + // exactly the same. + GetDependencyDigest() DependencyDigest + // GetRoutineOwner returns the username.SQLUsername of the routine's // (specified by routineOid) owner. GetRoutineOwner(ctx context.Context, routineOid oid.Oid) (username.SQLUsername, error) diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index 422c27e9765b..2029e911a3a0 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -1687,6 +1687,7 @@ func (f *ExprFmtCtx) formatColSimple(label string, id opt.ColumnID) { func (f *ExprFmtCtx) formatColSimpleToBuffer(buf *bytes.Buffer, label string, id opt.ColumnID) { if label == "" { if f.Memo != nil { + //nolint metadata will never be locked here. md := f.Memo.metadata fullyQualify := !f.HasFlags(ExprFmtHideQualifications) label = md.QualifiedAlias(f.Ctx, id, fullyQualify, false /* alwaysQualify */, f.Catalog) diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index 2fbb2c3f6d79..fa24b707c9e1 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -233,8 +233,9 @@ type Memo struct { // IsStale method for more details). func (m *Memo) Init(ctx context.Context, evalCtx *eval.Context) { // This initialization pattern ensures that fields are not unwittingly - // reused. Field reuse must be explicit. + // reused. Field reuse must be explicitpkg/sql/opt/memo/memo.go. *m = Memo{ + //nolint metadata is being reused. metadata: m.metadata, reorderJoinsLimit: int(evalCtx.SessionData().ReorderJoinsLimit), zigzagJoinEnabled: evalCtx.SessionData().ZigzagJoinEnabled, @@ -466,13 +467,12 @@ func (m *Memo) IsStale( m.txnIsoLevel != evalCtx.TxnIsoLevel { return true, nil } + // Memo is stale if the fingerprint of any object in the memo's metadata has // changed, or if the current user no longer has sufficient privilege to // access the object. - if depsUpToDate, err := m.Metadata().CheckDependencies(ctx, evalCtx, catalog); err != nil { + if depsUpToDate, err := m.Metadata().CheckDependencies(ctx, evalCtx, catalog); err != nil || !depsUpToDate { return true, err - } else if !depsUpToDate { - return true, nil } return false, nil } diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 91863b77aab6..58c55da97a27 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/intsets" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" "github.com/lib/pq/oid" ) @@ -141,6 +142,11 @@ type Metadata struct { // as a builtin function. builtinRefsByName map[tree.UnresolvedName]struct{} + digest struct { + syncutil.Mutex + depDigest cat.DependencyDigest + } + // NOTE! When adding fields here, update Init (if reusing allocated // data structures is desired), CopyFrom and TestMetadata. } @@ -361,6 +367,14 @@ func (md *Metadata) AddDependency(name MDDepName, ds cat.DataSource, priv privil } } +// dependencyDigestEquals checks if the stored dependency digest matches the +// current dependency digest. +func (md *Metadata) dependencyDigestEquals(currentDigest *cat.DependencyDigest) bool { + md.digest.Lock() + defer md.digest.Unlock() + return currentDigest.Equal(&md.digest.depDigest) +} + // CheckDependencies resolves (again) each database object on which this // metadata depends, in order to check the following conditions: // 1. The object has not been modified. @@ -378,6 +392,16 @@ func (md *Metadata) AddDependency(name MDDepName, ds cat.DataSource, priv privil func (md *Metadata) CheckDependencies( ctx context.Context, evalCtx *eval.Context, optCatalog cat.Catalog, ) (upToDate bool, err error) { + // If the query is AOST we must check all the dependencies, since the descriptors + // may have been different in the past. Otherwise, the dependency digest + // is sufficient. + currentDigest := optCatalog.GetDependencyDigest() + if evalCtx.AsOfSystemTime == nil && + !evalCtx.Txn.ReadTimestampFixed() && + md.dependencyDigestEquals(¤tDigest) { + return true, nil + } + // Check that no referenced data sources have changed. for id, dataSource := range md.dataSourceDeps { var toCheck cat.DataSource @@ -510,6 +534,11 @@ func (md *Metadata) CheckDependencies( } } + // Update the digest after a full dependency check, since our fast + // check did not succeed. + md.digest.Lock() + md.digest.depDigest = currentDigest + md.digest.Unlock() return true, nil } diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index e87a29bf15c8..977a9e2c67f8 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -47,9 +47,10 @@ const ( // Catalog implements the cat.Catalog interface for testing purposes. type Catalog struct { - testSchema Schema - counter int - enumTypes map[string]*types.T + testSchema Schema + counter int + dependencyDigest int64 + enumTypes map[string]*types.T udfs map[string]*tree.ResolvedFunctionDefinition revokedUDFOids intsets.Fast @@ -453,6 +454,15 @@ func (tc *Catalog) AddSequence(seq *Sequence) { tc.testSchema.dataSources[fq] = seq } +// GetDependencyDigest always assume that the generations are changing +// on us. +func (tc *Catalog) GetDependencyDigest() cat.DependencyDigest { + tc.dependencyDigest++ + return cat.DependencyDigest{ + LeaseGeneration: tc.dependencyDigest, + } +} + // ExecuteMultipleDDL parses the given semicolon-separated DDL SQL statements // and applies each of them to the test catalog. func (tc *Catalog) ExecuteMultipleDDL(sql string) error { diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 86545b97ec38..93c528a888b9 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -509,6 +509,24 @@ func (oc *optCatalog) GetCurrentUser() username.SQLUsername { return oc.planner.User() } +// GetDependencyDigest is part of the cat.Catalog interface. +func (oc *optCatalog) GetDependencyDigest() cat.DependencyDigest { + // The stats cache may not be setup in some tests like + // TestPortalsDestroyedOnTxnFinish. In which case always + // return the empty digest. + if oc.planner.ExecCfg().TableStatsCache == nil { + return cat.DependencyDigest{} + } + return cat.DependencyDigest{ + LeaseGeneration: oc.planner.Descriptors().GetLeaseGeneration(), + StatsGeneration: oc.planner.execCfg.TableStatsCache.GetGeneration(), + SystemConfig: oc.planner.execCfg.SystemConfig.GetSystemConfig(), + CurrentDatabase: oc.planner.CurrentDatabase(), + SearchPath: oc.planner.SessionData().SearchPath, + CurrentUser: oc.planner.User(), + } +} + // GetRoutineOwner is part of the cat.Catalog interface. func (oc *optCatalog) GetRoutineOwner( ctx context.Context, routineOid oid.Oid, diff --git a/pkg/sql/sessiondata/search_path.go b/pkg/sql/sessiondata/search_path.go index f19bb552ec84..7e2b15de037e 100644 --- a/pkg/sql/sessiondata/search_path.go +++ b/pkg/sql/sessiondata/search_path.go @@ -217,7 +217,7 @@ func (s SearchPath) Equals(other *SearchPath) bool { return false } // Fast path: skip the check if it is the same slice. - if &s.paths[0] != &other.paths[0] { + if len(s.paths) > 0 && &s.paths[0] != &other.paths[0] { for i := range s.paths { if s.paths[i] != other.paths[i] { return false diff --git a/pkg/sql/tests/BUILD.bazel b/pkg/sql/tests/BUILD.bazel index 9edd386d6c53..7596fda0b9af 100644 --- a/pkg/sql/tests/BUILD.bazel +++ b/pkg/sql/tests/BUILD.bazel @@ -35,6 +35,7 @@ go_test( "bank_test.go", "copy_file_upload_test.go", "deadlock_test.go", + "dependency_digest_test.go", "drop_role_concurrent_session_test.go", "empty_query_test.go", "enum_test.go", diff --git a/pkg/sql/tests/dependency_digest_test.go b/pkg/sql/tests/dependency_digest_test.go new file mode 100644 index 000000000000..69906d713e95 --- /dev/null +++ b/pkg/sql/tests/dependency_digest_test.go @@ -0,0 +1,147 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package tests + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +// TestDependencyDigestOptimization validates that dependency digest information +// is properly invalidated in the face of modifications for prepared queries. +func TestDependencyDigestOptimization(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + c := serverutils.StartCluster(t, 3, base.TestClusterArgs{}) + + changeRawConn := c.ServerConn(0) + prepareRawConn := c.ServerConn(1) + + changeConn := sqlutils.MakeSQLRunner(changeRawConn) + prepareConn := sqlutils.MakeSQLRunner(prepareRawConn) + + // Setup a database with multiple schemas. + changeConn.ExecMultiple(t, + "CREATE USER bob", + "CREATE DATABASE db1", + "CREATE DATABASE db2", + "CREATE SCHEMA db1.sc1", + "CREATE SCHEMA db1.sc2", + "CREATE SCHEMA db1.sc3", + "CREATE SCHEMA db2.sc3", + "CREATE TABLE db2.sc3.t1(m int)", + "CREATE TABLE db1.sc3.t1(n int)", + "CREATE TYPE IF NOT EXISTS db1.sc3.status AS ENUM ('open', 'closed', 'inactive');", + "USE db1", + `CREATE FUNCTION db1.sc3.f1 (input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT 1; + $$;`, + "USE db2", + `CREATE FUNCTION db2.sc3.f1 (input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT 32; + $$;`, + "INSERT INTO db1.sc3.t1 VALUES(1)", + "INSERT INTO db2.sc3.t1 VALUES(32)", + ) + + // Set up the search_path and a simple select against t1. + prepareConn.Exec(t, "USE db1") + prepareConn.Exec(t, "SET search_path=sc1,sc2,sc4,sc3") + query, err := prepareRawConn.Prepare("SELECT * FROM t1") + require.NoError(t, err) + + // Confirm the table from sc3 is being selected. + validateColumnsOnQuery := func(exp int) { + row := query.QueryRow() + require.NoError(t, row.Err()) + var val int + require.NoError(t, row.Scan(&val)) + require.Equal(t, exp, val) + // Validate the function query next. + tRow := prepareConn.QueryRow(t, "SELECT f1(32)") + tRow.Scan(&val) + require.Equal(t, exp, val) + } + // Sanity: Ensure that sc3.t1 is picked in the original prepare. + validateColumnsOnQuery(1) + // Sanity: Ensure the type resolution works. + prepareConn.Exec(t, "SELECT 'open'::STATUS") + // Set up a new t1/f1 in sc2, which should be picked earlier in our search_path. + changeConn.Exec(t, "CREATE TABLE db1.sc2.t1(k int)") + changeConn.Exec(t, "INSERT INTO db1.sc2.t1 VALUES(2)") + changeConn.Exec(t, ` +USE db1; +CREATE FUNCTION db1.sc2.f1 (input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT 2; + $$;`) + // Confirm the table from sc2 is being selected next. + validateColumnsOnQuery(2) + // Setup a new type that will not have open. + changeConn.Exec(t, "CREATE TYPE IF NOT EXISTS db1.sc2.status AS ENUM ('closed', 'inactive');") + prepareConn.ExpectErr(t, "invalid input value for enum status: \"open\"", + + "SELECT 'open'::STATUS") + // Alter table db1.sc2.t1 to only have a virtual column + changeConn.Exec(t, "ALTER TABLE db1.sc2.t1 DROP COLUMN k, ADD COLUMN n INT AS (5) VIRTUAL") + // Replace the function to have a new value + changeConn.Exec(t, ` + CREATE OR REPLACE FUNCTION db1.sc2.f1 (input INT) RETURNS INT8 + LANGUAGE SQL + AS $$ + SELECT 5; + $$;`) + validateColumnsOnQuery(5) + // Modify the search_path so that sc3 is first, and validate the prepared + // query is properly updated. + prepareConn.Exec(t, "SET search_path=sc3,sc2,sc4,sc1") + validateColumnsOnQuery(1) + // Change our database path next, we should select db2.sc3.t1. + prepareConn.Exec(t, "USE db2") + validateColumnsOnQuery(32) + // Confirm that statistics changes have the same behavior. + prepareConn.Exec(t, "PREPARE p1 AS (SELECT * FROM t1)") + explainRows := prepareConn.QueryStr(t, "EXPLAIN ANALYZE EXECUTE p1") + foundReusedPlan := false + for _, row := range explainRows { + // Same as the prepare above, so we don't expect any changes. + if row[0] == "plan type: generic, reused" { + foundReusedPlan = true + break + } + } + require.Truef(t, foundReusedPlan, "did not find reused plan") + changeConn.Exec(t, "CREATE STATISTICS s1 FROM db2.sc3.t1") + explainRows = prepareConn.QueryStr(t, "EXPLAIN ANALYZE EXECUTE p1") + foundReOptimizedPlan := false + for _, row := range explainRows { + // New stats were generated so the plan should get re-optimized. + if row[0] == "plan type: generic, re-optimized" { + foundReOptimizedPlan = true + break + } + } + require.Truef(t, foundReOptimizedPlan, "did not find re-optimized plan") + // Confirm changing the role will have trouble. + prepareConn.Exec(t, "SET ROLE bob") + row := query.QueryRow() + require.Errorf(t, row.Err(), "user bob does not have USAGE privilege on schema sc3") + defer c.Stopper().Stop(context.Background()) + +} From 3c749bc783f4558c46e7f02df383fcd0a7ac93e6 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Fri, 20 Dec 2024 08:46:29 -0500 Subject: [PATCH 075/126] sql: add catalog_digest_staleness_check_enabled to control optimization Previously, we added support for checking memo staleness using dependency digests. We currently have no way of disabling this check if something goes wrong. To address this, this patch adds catalog_digest_staleness_check_enabled for skipping fast memo staleness checks. Release note: None --- pkg/sql/exec_util.go | 4 ++++ pkg/sql/opt/metadata.go | 11 +++++++---- .../sessiondatapb/local_only_session_data.proto | 3 +++ pkg/sql/vars.go | 17 +++++++++++++++++ 4 files changed, 31 insertions(+), 4 deletions(-) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index d82671031c1e..6119f0899d21 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -4001,6 +4001,10 @@ func (m *sessionDataMutator) SetLegacyVarcharTyping(val bool) { m.data.LegacyVarcharTyping = val } +func (m *sessionDataMutator) SetCatalogDigestStalenessCheckEnabled(b bool) { + m.data.CatalogDigestStalenessCheckEnabled = b +} + // Utility functions related to scrubbing sensitive information on SQL Stats. // quantizeCounts ensures that the Count field in the diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 58c55da97a27..ff4b7cae24c3 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -396,7 +396,8 @@ func (md *Metadata) CheckDependencies( // may have been different in the past. Otherwise, the dependency digest // is sufficient. currentDigest := optCatalog.GetDependencyDigest() - if evalCtx.AsOfSystemTime == nil && + if evalCtx.SessionData().CatalogDigestStalenessCheckEnabled && + evalCtx.AsOfSystemTime == nil && !evalCtx.Txn.ReadTimestampFixed() && md.dependencyDigestEquals(¤tDigest) { return true, nil @@ -536,9 +537,11 @@ func (md *Metadata) CheckDependencies( // Update the digest after a full dependency check, since our fast // check did not succeed. - md.digest.Lock() - md.digest.depDigest = currentDigest - md.digest.Unlock() + if evalCtx.SessionData().CatalogDigestStalenessCheckEnabled { + md.digest.Lock() + md.digest.depDigest = currentDigest + md.digest.Unlock() + } return true, nil } diff --git a/pkg/sql/sessiondatapb/local_only_session_data.proto b/pkg/sql/sessiondatapb/local_only_session_data.proto index a529d80b0524..f07b3c86e0da 100644 --- a/pkg/sql/sessiondatapb/local_only_session_data.proto +++ b/pkg/sql/sessiondatapb/local_only_session_data.proto @@ -591,6 +591,9 @@ message LocalOnlySessionData { bool avoid_full_table_scans_in_mutations = 151; // RowLevelSecurityEnabled indicates whether row level security is enabled bool row_level_security_enabled = 152; + // CatalogDigestStalenessCheckEnabled is used to enable using the catalog + // digest information to do fast memo checks. + bool catalog_digest_staleness_check_enabled = 153; /////////////////////////////////////////////////////////////////////////// // WARNING: consider whether a session parameter you're adding needs to // diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 5eae1fd5e152..b4456217ca82 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -3724,6 +3724,23 @@ var varGen = map[string]sessionVar{ }, GlobalDefault: globalFalse, }, + // CockroachDB extension. + `catalog_digest_staleness_check_enabled`: { + GetStringVal: makePostgresBoolGetStringValFn(`catalog_digest_staleness_check_enabled`), + Set: func(_ context.Context, m sessionDataMutator, s string) error { + b, err := paramparse.ParseBoolVar("catalog_digest_staleness_check_enabled", s) + if err != nil { + return err + } + m.SetCatalogDigestStalenessCheckEnabled(b) + return nil + }, + Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) { + return formatBoolAsPostgresSetting(evalCtx.SessionData().CatalogDigestStalenessCheckEnabled), nil + }, + GlobalDefault: globalTrue, + Hidden: true, + }, } func ReplicationModeFromString(s string) (sessiondatapb.ReplicationMode, error) { From 926f5ed1bc88c157750efe59bbd81e29e5a5c303 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Mon, 13 Jan 2025 15:59:17 -0500 Subject: [PATCH 076/126] roachtest: update activerecord and ruby-g expected failures - Mark 2 activerecord tests as flaky. - Mark 4 ruby-pg tests as passing after https://github.com/cockroachdb/cockroach/pull/138709 was merged. Release note: None --- pkg/cmd/roachtest/tests/activerecord_blocklist.go | 2 ++ pkg/cmd/roachtest/tests/ruby_pg_blocklist.go | 4 ---- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/pkg/cmd/roachtest/tests/activerecord_blocklist.go b/pkg/cmd/roachtest/tests/activerecord_blocklist.go index 749a22d4189a..b2905dba53db 100644 --- a/pkg/cmd/roachtest/tests/activerecord_blocklist.go +++ b/pkg/cmd/roachtest/tests/activerecord_blocklist.go @@ -27,6 +27,8 @@ var activeRecordIgnoreList = blocklist{ `BasicsTest#test_default_values_are_deeply_dupped`: "flaky", `CockroachDB::FixturesTest#test_create_fixtures`: "flaky", `FixtureWithSetModelClassPrevailsOverNamingConventionTest#test_model_class_in_fixture_file_is_respected`: "flaky", + `InheritanceTest#test_eager_load_belongs_to_primary_key_quoting`: "flaky", + `InheritanceTest#test_eager_load_belongs_to_something_inherited`: "flaky", `TestAutosaveAssociationOnAHasAndBelongsToManyAssociation#test_should_not_save_and_return_false_if_a_callback_cancelled_saving_in_either_create_or_update`: "flaky", `TestAutosaveAssociationOnAHasAndBelongsToManyAssociation#test_should_not_update_children_when_parent_creation_with_no_reason`: "flaky", `TestAutosaveAssociationOnAHasAndBelongsToManyAssociation#test_should_update_children_when_autosave_is_true_and_parent_is_new_but_child_is_not`: "flaky", diff --git a/pkg/cmd/roachtest/tests/ruby_pg_blocklist.go b/pkg/cmd/roachtest/tests/ruby_pg_blocklist.go index 20e45c8fda06..70edd47607f3 100644 --- a/pkg/cmd/roachtest/tests/ruby_pg_blocklist.go +++ b/pkg/cmd/roachtest/tests/ruby_pg_blocklist.go @@ -54,8 +54,6 @@ var rubyPGBlocklist = blocklist{ `PG::Connection multinationalization support Ruby 1.9.x default_internal encoding allows users of the async interface to set the client_encoding to the default_internal`: "unknown", `PG::Connection multinationalization support Ruby 1.9.x default_internal encoding honors the Encoding.default_internal if it's set and the synchronous interface is used`: "unknown", `PG::Connection multinationalization support encodes exception messages with the connection's encoding (#96)`: "unknown", - `PG::Connection multinationalization support handles clearing result in or after set_notice_receiver`: "unknown", - `PG::Connection multinationalization support receives properly encoded messages in the notice callbacks`: "unknown", `PG::Connection multinationalization support receives properly encoded text from wait_for_notify`: "unknown", `PG::Connection multinationalization support respect and convert character encoding of input strings should convert query string and parameters to #exec_params`: "unknown", `PG::Connection multinationalization support respect and convert character encoding of input strings should convert query string and parameters to #send_query_params`: "unknown", @@ -132,8 +130,6 @@ var rubyPGBlocklist = blocklist{ `running with sync_* methods PG::Connection multinationalization support Ruby 1.9.x default_internal encoding allows users of the async interface to set the client_encoding to the default_internal`: "unknown", `running with sync_* methods PG::Connection multinationalization support Ruby 1.9.x default_internal encoding honors the Encoding.default_internal if it's set and the synchronous interface is used`: "unknown", `running with sync_* methods PG::Connection multinationalization support encodes exception messages with the connection's encoding (#96)`: "unknown", - `running with sync_* methods PG::Connection multinationalization support handles clearing result in or after set_notice_receiver`: "unknown", - `running with sync_* methods PG::Connection multinationalization support receives properly encoded messages in the notice callbacks`: "unknown", `running with sync_* methods PG::Connection multinationalization support receives properly encoded text from wait_for_notify`: "unknown", `running with sync_* methods PG::Connection multinationalization support respect and convert character encoding of input strings should convert query string and parameters to #exec_params`: "unknown", `running with sync_* methods PG::Connection multinationalization support respect and convert character encoding of input strings should convert query string and parameters to #send_query_params`: "unknown", From 442912623df65b76fae048a884447525d29361b2 Mon Sep 17 00:00:00 2001 From: Kyle Wong <37189875+kyle-a-wong@users.noreply.github.com> Date: Mon, 13 Jan 2025 16:30:40 -0500 Subject: [PATCH 077/126] roachtest: deflake db-console/mixed-version-cypress This roachtest periodically flakes due to errors when building the docker image which runs cypress tests. This commit wraps it in a testutils.SuccessSoon to retry when the build fails Fixes: #138884 Epic: None Release note: None --- pkg/cmd/roachtest/tests/db_console.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/pkg/cmd/roachtest/tests/db_console.go b/pkg/cmd/roachtest/tests/db_console.go index f1c9a611da09..195f6134e2c1 100644 --- a/pkg/cmd/roachtest/tests/db_console.go +++ b/pkg/cmd/roachtest/tests/db_console.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/testutils" e2e_tests "github.com/cockroachdb/cockroach/pkg/ui/workspaces/e2e-tests" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -158,8 +159,10 @@ func (d *dbConsoleCypressTest) buildDockerImage(ctx context.Context) { require.NoError(t, rtCluster.Install(ctx, t.L(), workloadNode, "docker"), "failed to install docker") // Build docker image on the workload node. - require.NoError(t, rtCluster.RunE(ctx, option.WithNodes(workloadNode), - fmt.Sprintf("docker build -t %s %s", d.imageName, d.cypressWorkingDir))) + testutils.SucceedsSoon(t, func() error { + return rtCluster.RunE(ctx, option.WithNodes(workloadNode), + fmt.Sprintf("docker build -t %s %s", d.imageName, d.cypressWorkingDir)) + }) } // writeCypressFilesToWorkloadNode writes the embedded dbConsoleCypressTest.cypressFiles to the From b0f2b656dfa2df9a182d32b1e72af4824680e453 Mon Sep 17 00:00:00 2001 From: DarrylWong Date: Thu, 9 Jan 2025 13:55:32 -0500 Subject: [PATCH 078/126] roachprod: retry hostname validation on different nodes Previously hostname validation was only attempted on the first node. If the first node reports a mismatch in names, the entire cluster should mismatch as well. However, it's possible that the first node fails due to a transient error. In this case, we want to try other nodes as a VM being down does not mean the hostnames are wrong. This is an important distinction for artifacts collection. If a test failed because the a VM went down, we still want to collect logs from the other nodes. --- pkg/roachprod/install/cluster_synced.go | 38 +++++++++++++++++++------ 1 file changed, 30 insertions(+), 8 deletions(-) diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index 82a19588cb93..d705e171575b 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -369,16 +369,38 @@ if ! %s; then `, isValidHost, errMsg, elseBranch) } -// validateHost will run `validateHostnameCmd` on the node passed to +// validateHost will run `validateHostnameCmd` on the node(s) passed to // make sure it still belongs to the SyncedCluster. Returns an error // when the hostnames don't match, indicating that the roachprod cache // is stale. -func (c *SyncedCluster) validateHost(ctx context.Context, l *logger.Logger, node Node) error { +func (c *SyncedCluster) validateHost(ctx context.Context, l *logger.Logger, nodes Nodes) error { if c.IsLocal() { return nil } - cmd := c.validateHostnameCmd("", node) - return c.Run(ctx, l, l.Stdout, l.Stderr, WithNodes(Nodes{node}), "validate-ssh-host", cmd) + + // Retry on different nodes, in case some of the VMs are unreachable. + // While this does indicate something is likely wrong, we don't want to + // fail here as some callers want to tolerate this, e.g. fetching logs + // after a test failure shouldn't fail just because one VM is down. + var combinedErr error + retryOpts := *DefaultRetryOpt + retryOpts.MaxRetries = 4 + r := retry.StartWithCtx(ctx, retryOpts) + for nodeIdx := 0; r.Next(); nodeIdx = (nodeIdx + 1) % len(nodes) { + node := nodes[nodeIdx] + cmd := c.validateHostnameCmd("", node) + + err := c.Run(ctx, l, l.Stdout, l.Stderr, WithNodes(Nodes{node}).WithRetryDisabled(), "validate-ssh-host", cmd) + if err != nil { + if !rperrors.IsTransient(err) { + return err + } + combinedErr = errors.CombineErrors(combinedErr, err) + continue + } + return nil + } + return combinedErr } // cmdDebugName is the suffix of the generated ssh debug file @@ -1641,7 +1663,7 @@ func (c *SyncedCluster) PutString( func (c *SyncedCluster) Put( ctx context.Context, l *logger.Logger, nodes Nodes, src string, dest string, ) error { - if err := c.validateHost(ctx, l, nodes[0]); err != nil { + if err := c.validateHost(ctx, l, nodes); err != nil { return err } // Check if source file exists and if it's a symlink. @@ -1885,7 +1907,7 @@ func (c *SyncedCluster) Logs( from, to time.Time, out io.Writer, ) error { - if err := c.validateHost(context.TODO(), l, c.Nodes[0]); err != nil { + if err := c.validateHost(context.TODO(), l, c.Nodes); err != nil { return err } rsyncNodeLogs := func(ctx context.Context, node Node) error { @@ -2013,7 +2035,7 @@ func (c *SyncedCluster) Logs( func (c *SyncedCluster) Get( ctx context.Context, l *logger.Logger, nodes Nodes, src, dest string, ) error { - if err := c.validateHost(context.TODO(), l, nodes[0]); err != nil { + if err := c.validateHost(ctx, l, nodes); err != nil { return err } // TODO(peter): Only get 10 nodes at a time. When a node completes, output a @@ -2281,7 +2303,7 @@ func (c *SyncedCluster) loadBalancerURL( // exclusively. func (c *SyncedCluster) SSH(ctx context.Context, l *logger.Logger, sshArgs, args []string) error { targetNode := c.Nodes[0] - if err := c.validateHost(ctx, l, targetNode); err != nil { + if err := c.validateHost(ctx, l, Nodes{targetNode}); err != nil { return err } From 067579e89be31fb152887d3e2babd4a950f9365a Mon Sep 17 00:00:00 2001 From: Matt White Date: Fri, 13 Dec 2024 14:42:45 -0800 Subject: [PATCH 079/126] vecstore: implement partition manipulation functions Add persistent storage to vecstore. Implement the methods for partition creation, retrieval and deletion. Add a simple test to ensure these methods are functioning. Epic: CRDB-42943 --- pkg/sql/vecindex/vecstore/BUILD.bazel | 16 + pkg/sql/vecindex/vecstore/encoding.go | 6 + pkg/sql/vecindex/vecstore/main_test.go | 6 + pkg/sql/vecindex/vecstore/persistent_store.go | 64 ++++ .../vecstore/persistent_store_test.go | 93 +++++ pkg/sql/vecindex/vecstore/persistent_txn.go | 322 ++++++++++++++++++ 6 files changed, 507 insertions(+) create mode 100644 pkg/sql/vecindex/vecstore/persistent_store.go create mode 100644 pkg/sql/vecindex/vecstore/persistent_store_test.go create mode 100644 pkg/sql/vecindex/vecstore/persistent_txn.go diff --git a/pkg/sql/vecindex/vecstore/BUILD.bazel b/pkg/sql/vecindex/vecstore/BUILD.bazel index 5b3986a3cfb6..20c8e09dc071 100644 --- a/pkg/sql/vecindex/vecstore/BUILD.bazel +++ b/pkg/sql/vecindex/vecstore/BUILD.bazel @@ -33,6 +33,8 @@ go_library( "in_memory_store.go", "in_memory_txn.go", "partition.go", + "persistent_store.go", + "persistent_txn.go", "search_set.go", "store.go", "vecstorepb.go", @@ -41,6 +43,11 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/vecindex/vecstore", visibility = ["//visibility:public"], deps = [ + "//pkg/kv", + "//pkg/kv/kvpb", + "//pkg/kv/kvserver/concurrency/isolation", + "//pkg/roachpb", + "//pkg/sql/sem/builtins", "//pkg/sql/vecindex/internal", "//pkg/sql/vecindex/quantize", "//pkg/util/container/heap", @@ -61,16 +68,25 @@ go_test( "in_memory_store_test.go", "main_test.go", "partition_test.go", + "persistent_store_test.go", "search_set_test.go", "vecstorepb_test.go", ], embed = [":vecstore"], deps = [ + "//pkg/base", + "//pkg/keys", + "//pkg/roachpb", + "//pkg/security/securityassets", + "//pkg/security/securitytest", + "//pkg/server", "//pkg/sql/randgen", + "//pkg/sql/rowenc", "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/sql/vecindex/internal", "//pkg/sql/vecindex/quantize", + "//pkg/testutils/serverutils", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/num32", diff --git a/pkg/sql/vecindex/vecstore/encoding.go b/pkg/sql/vecindex/vecstore/encoding.go index 0e9745b8e36d..d92fc82ce724 100644 --- a/pkg/sql/vecindex/vecstore/encoding.go +++ b/pkg/sql/vecindex/vecstore/encoding.go @@ -49,6 +49,12 @@ func EncodePartitionKey(appendTo []byte, key PartitionKey) []byte { return encoding.EncodeUvarintAscending(appendTo, uint64(key)) } +// EncodedPartitionKeyLen returns the number of bytes needed to encode the +// partition key. +func EncodedPartitionKeyLen(key PartitionKey) int { + return encoding.EncLenUvarintAscending(uint64(key)) +} + // EncodeChildKey encodes a child key into the given byte slice. The "appendTo" // slice is expected to be the prefix shared between all KV entries for a // partition. diff --git a/pkg/sql/vecindex/vecstore/main_test.go b/pkg/sql/vecindex/vecstore/main_test.go index e305c8b99667..0f6765d2f94b 100644 --- a/pkg/sql/vecindex/vecstore/main_test.go +++ b/pkg/sql/vecindex/vecstore/main_test.go @@ -9,13 +9,19 @@ import ( "os" "testing" + "github.com/cockroachdb/cockroach/pkg/security/securityassets" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/randutil" ) //go:generate ../util/leaktest/add-leaktest.sh *_test.go func TestMain(m *testing.M) { + securityassets.SetLoader(securitytest.EmbeddedAssets) randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) os.Exit(m.Run()) } diff --git a/pkg/sql/vecindex/vecstore/persistent_store.go b/pkg/sql/vecindex/vecstore/persistent_store.go new file mode 100644 index 000000000000..5a44bb3b9c46 --- /dev/null +++ b/pkg/sql/vecindex/vecstore/persistent_store.go @@ -0,0 +1,64 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package vecstore + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/vecindex/quantize" +) + +// PersistentStore implements the Store interface for KV backed vector indices. +type PersistentStore struct { + db *kv.DB // Needed for index maintenance functions + quantizer quantize.Quantizer + rootQuantizer quantize.Quantizer + + prefix roachpb.Key +} + +var _ Store = (*PersistentStore)(nil) + +// NewPersistentStore creates a vecstore.Store interface backed by the KV for a +// single vector index. +func NewPersistentStore( + db *kv.DB, quantizer quantize.Quantizer, prefix roachpb.Key, +) *PersistentStore { + ps := PersistentStore{ + db: db, + quantizer: quantizer, + rootQuantizer: quantize.NewUnQuantizer(quantizer.GetOriginalDims()), + prefix: prefix, + } + + return &ps +} + +// Begin is part of the vecstore.Store interface. Begin creates a new KV +// transaction on behalf of the user and prepares it to operate on the persistent +// vector store. +func (s *PersistentStore) Begin(ctx context.Context) (Txn, error) { + return NewPersistentStoreTxn(s, s.db.NewTxn(ctx, "vecstore.PersistentStore begin transaction")), nil +} + +// Commit is part of the vecstore.Store interface. Commit commits the +// underlying KV transaction wrapped by the vecstore.Txn passed in. +func (s *PersistentStore) Commit(ctx context.Context, txn Txn) error { + return txn.(*persistentStoreTxn).kv.Commit(ctx) +} + +// Abort is part of the vecstore.Store interface. Abort causes the underlying +// KV transaction wrapped by the passed vecstore.Txn to roll back. +func (s *PersistentStore) Abort(ctx context.Context, txn Txn) error { + return txn.(*persistentStoreTxn).kv.Rollback(ctx) +} + +// MergeStats is part of the vecstore.Store interface. +func (s *PersistentStore) MergeStats(ctx context.Context, stats *IndexStats, skipMerge bool) error { + panic("MergeStats() unimplemented") +} diff --git a/pkg/sql/vecindex/vecstore/persistent_store_test.go b/pkg/sql/vecindex/vecstore/persistent_store_test.go new file mode 100644 index 000000000000..33ffb821951e --- /dev/null +++ b/pkg/sql/vecindex/vecstore/persistent_store_test.go @@ -0,0 +1,93 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package vecstore + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/vecindex/internal" + "github.com/cockroachdb/cockroach/pkg/sql/vecindex/quantize" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/vector" + "github.com/stretchr/testify/require" +) + +func TestPersistentStore(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := internal.WithWorkspace(context.Background(), &internal.Workspace{}) + s, _, kvDB := serverutils.StartServer(t, base.TestServerArgs{DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(42)}) + defer s.Stopper().Stop(ctx) + + childKey2 := ChildKey{PartitionKey: 2} + childKey10 := ChildKey{PartitionKey: 10} + childKey20 := ChildKey{PartitionKey: 20} + primaryKey200 := ChildKey{PrimaryKey: PrimaryKey{2, 00}} + primaryKey300 := ChildKey{PrimaryKey: PrimaryKey{3, 00}} + primaryKey400 := ChildKey{PrimaryKey: PrimaryKey{4, 00}} + + ten5Codec := keys.MakeSQLCodec(roachpb.MustMakeTenantID(5)) + prefix := rowenc.MakeIndexKeyPrefix(ten5Codec, 500, 42) + quantizer := quantize.NewUnQuantizer(2) + store := NewPersistentStore(kvDB, quantizer, prefix) + + t.Run("insert a root partition into the store and read it back", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + vectors := vector.T{4, 3}.AsSet() + quantizedSet := quantizer.Quantize(ctx, &vectors) + root := NewPartition(quantizer, quantizedSet, []ChildKey{childKey2}, Level(2)) + require.NoError(t, txn.SetRootPartition(ctx, root)) + readRoot, err := txn.GetPartition(ctx, RootKey) + require.NoError(t, err) + testingAssertPartitionsEqual(t, root, readRoot) + + vectors = vector.T{4, 3}.AsSet() + vectors.Add(vector.T{2, 1}) + quantizedSet = quantizer.Quantize(ctx, &vectors) + root = NewPartition(quantizer, quantizedSet, []ChildKey{childKey10, childKey20}, Level(2)) + require.NoError(t, txn.SetRootPartition(ctx, root)) + readRoot, err = txn.GetPartition(ctx, RootKey) + require.NoError(t, err) + testingAssertPartitionsEqual(t, root, readRoot) + + vectors = vector.T{4, 3}.AsSet() + vectors.Add(vector.T{2, 1}) + vectors.Add(vector.T{5, 6}) + quantizedSet = quantizer.Quantize(ctx, &vectors) + root = NewPartition(quantizer, quantizedSet, []ChildKey{primaryKey200, primaryKey300, primaryKey400}, LeafLevel) + require.NoError(t, txn.SetRootPartition(ctx, root)) + readRoot, err = txn.GetPartition(ctx, RootKey) + require.NoError(t, err) + testingAssertPartitionsEqual(t, root, readRoot) + }) + + t.Run("insert a partition and then delete it", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + vectors := vector.T{4, 3}.AsSet() + quantizedSet := quantizer.Quantize(ctx, &vectors) + testPartition := NewPartition(quantizer, quantizedSet, []ChildKey{childKey2}, Level(2)) + partitionKey, err := txn.InsertPartition(ctx, testPartition) + require.NoError(t, err) + newPartition, err := txn.GetPartition(ctx, partitionKey) + require.NoError(t, err) + testingAssertPartitionsEqual(t, testPartition, newPartition) + + err = txn.DeletePartition(ctx, partitionKey) + require.NoError(t, err) + _, err = txn.GetPartition(ctx, partitionKey) + require.Error(t, err) + }) +} diff --git a/pkg/sql/vecindex/vecstore/persistent_txn.go b/pkg/sql/vecindex/vecstore/persistent_txn.go new file mode 100644 index 000000000000..4326206b666b --- /dev/null +++ b/pkg/sql/vecindex/vecstore/persistent_txn.go @@ -0,0 +1,322 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package vecstore + +import ( + "context" + "slices" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" + "github.com/cockroachdb/cockroach/pkg/sql/vecindex/quantize" + "github.com/cockroachdb/cockroach/pkg/util/vector" + "github.com/cockroachdb/errors" +) + +// persistentStoreTxn provides a context to make transactional changes to a +// vector index. Calling methods here will use the wrapped KV Txn to update the +// vector index's internal data. Committing changes is the responsibility of the +// caller. +type persistentStoreTxn struct { + kv *kv.Txn + store *PersistentStore + + lockDurability kvpb.KeyLockingDurabilityType + tmpChildKeys []ChildKey + + codec persistentStoreCodec + rootCodec persistentStoreCodec +} + +var _ Txn = (*persistentStoreTxn)(nil) + +// persistentStoreCodec abstracts quantizer specific encode/decode operations +// from the rest of the persistent store. +type persistentStoreCodec struct { + quantizer quantize.Quantizer + tmpVectorSet quantize.QuantizedVectorSet +} + +// newPersistentStoreCodec creates a new PersistentStoreCodec wrapping the +// provided quantizer. +func newPersistentStoreCodec(quantizer quantize.Quantizer) persistentStoreCodec { + return persistentStoreCodec{ + quantizer: quantizer, + } +} + +// clear resets the codec's internal vector set to start a new encode / decode +// operation. +func (psc *persistentStoreCodec) clear(minCapacity int, centroid vector.T) { + if psc.tmpVectorSet == nil { + psc.tmpVectorSet = psc.quantizer.NewQuantizedVectorSet(minCapacity, centroid) + } else { + psc.tmpVectorSet.Clear(centroid) + } +} + +// getVectorSet returns the internal vector set cache. These will be invalidated +// when the clear method is called. +func (psc *persistentStoreCodec) getVectorSet() quantize.QuantizedVectorSet { + return psc.tmpVectorSet +} + +// decodeVector decodes a single vector to the codec's internal vector set. +func (psc *persistentStoreCodec) decodeVector(encodedVector []byte) error { + switch psc.quantizer.(type) { + case *quantize.UnQuantizer: + return DecodeUnquantizedVectorToSet(encodedVector, psc.tmpVectorSet.(*quantize.UnQuantizedVectorSet)) + case *quantize.RaBitQuantizer: + return DecodeRaBitQVectorToSet(encodedVector, psc.tmpVectorSet.(*quantize.RaBitQuantizedVectorSet)) + } + return errors.Errorf("unknown quantizer type %T", psc.quantizer) +} + +// encodeVector encodes a single vector. This method invalidates the internal +// vector set. +func (psc *persistentStoreCodec) encodeVector( + ctx context.Context, v vector.T, centroid vector.T, +) ([]byte, error) { + psc.clear(1, centroid) + input := v.AsSet() + psc.quantizer.QuantizeInSet(ctx, psc.tmpVectorSet, &input) + return psc.encodeVectorFromSet(psc.tmpVectorSet, 0 /* idx */) +} + +// encodeVectorFromSet encodes the vector indicated by 'idx' from an external +// vector set. +func (psc *persistentStoreCodec) encodeVectorFromSet( + vs quantize.QuantizedVectorSet, idx int, +) ([]byte, error) { + switch psc.quantizer.(type) { + case *quantize.UnQuantizer: + dist := vs.GetCentroidDistances() + return EncodeUnquantizedVector([]byte{}, dist[idx], vs.(*quantize.UnQuantizedVectorSet).Vectors.At(idx)) + case *quantize.RaBitQuantizer: + dist := psc.tmpVectorSet.GetCentroidDistances() + qs := psc.tmpVectorSet.(*quantize.RaBitQuantizedVectorSet) + return EncodeRaBitQVector( + []byte{}, + qs.CodeCounts[idx], + dist[idx], + qs.DotProducts[idx], + qs.Codes.At(idx), + ), nil + } + return nil, errors.Errorf("unknown quantizer type %T", psc.quantizer) +} + +// NewPersistentStoreTxn wraps a PersistentStore transaction around a kv +// transaction for use with the vecstore API. +func NewPersistentStoreTxn(store *PersistentStore, kv *kv.Txn) *persistentStoreTxn { + psTxn := persistentStoreTxn{ + kv: kv, + store: store, + codec: newPersistentStoreCodec(store.quantizer), + rootCodec: newPersistentStoreCodec(store.rootQuantizer), + } + // TODO (mw5h): This doesn't take into account session variables that control + // lock durability. This doesn't matter for partition maintenance operations that + // don't have a session, but may lead to unexpected behavior for CRUD operations. + // The logic for determining what to do there is in optBuilder, so there may be + // some plumbing involved to get it down here. + if kv.IsoLevel() == isolation.Serializable { + psTxn.lockDurability = kvpb.BestEffort + } else { + psTxn.lockDurability = kvpb.GuaranteedDurability + } + + return &psTxn +} + +// getCodecForPartitionKey returns the correct codec to use for interacting with +// the partition indicated. This will be the unquantized codec for the root +// partition, the codec for the quantizer indicated when the store was created +// otherwise. +func (psTxn *persistentStoreTxn) getCodecForPartitionKey( + partitionKey PartitionKey, +) *persistentStoreCodec { + // We always store the full sized vectors in the root partition + if partitionKey == RootKey { + return &psTxn.rootCodec + } else { + return &psTxn.codec + } +} + +// decodePartition decodes the KV row set into an ephemeral partition. This +// partition will become invalid when the codec is next reset, so it needs to be +// cloned if it will be used outside of the persistent store. +func (psTxn *persistentStoreTxn) decodePartition( + codec *persistentStoreCodec, result *kv.Result, +) (*Partition, error) { + if result.Err != nil { + return nil, result.Err + } + if len(result.Rows) == 0 { + return nil, ErrPartitionNotFound + } + + // Partition metadata is stored in /Prefix/PartitionID, with vector data + // following in /Prefix/PartitionID/ChildKey. + level, centroid, err := DecodePartitionMetadata(result.Rows[0].ValueBytes()) + if err != nil { + return nil, err + } + metaKeyLen := len(result.Rows[0].Key) + vectorEntries := result.Rows[1:] + + // Clear and ensure storage for the vector entries and child keys. + codec.clear(len(vectorEntries), centroid) + if cap(psTxn.tmpChildKeys) < len(vectorEntries) { + psTxn.tmpChildKeys = slices.Grow(psTxn.tmpChildKeys, len(vectorEntries)-cap(psTxn.tmpChildKeys)) + } + psTxn.tmpChildKeys = psTxn.tmpChildKeys[:len(vectorEntries)] + + for i, entry := range vectorEntries { + childKey, err := DecodeChildKey(entry.Key[metaKeyLen:], level) + if err != nil { + return nil, err + } + psTxn.tmpChildKeys[i] = childKey + + if err = codec.decodeVector(entry.ValueBytes()); err != nil { + return nil, err + } + } + + return NewPartition(codec.quantizer, codec.getVectorSet(), psTxn.tmpChildKeys, level), nil +} + +// GetPartition is part of the vecstore.Txn interface. Read the partition +// indicated by `partitionKey` and build a Partition data structure, which is +// returned. +func (psTxn *persistentStoreTxn) GetPartition( + ctx context.Context, partitionKey PartitionKey, +) (*Partition, error) { + b := psTxn.kv.NewBatch() + + startKey := psTxn.encodePartitionKey(partitionKey) + endKey := startKey.PrefixEnd() + + // GetPartition is used by fixup to split and merge partitions, so we want to + // block concurrent writes. + b.ScanForUpdate(startKey, endKey, psTxn.lockDurability) + err := psTxn.kv.Run(ctx, b) + if err != nil { + return nil, err + } + + codec := psTxn.getCodecForPartitionKey(partitionKey) + partition, err := psTxn.decodePartition(codec, &b.Results[0]) + if err != nil { + return nil, err + } + return partition.Clone(), nil +} + +// Insert a partition with the given partition key into the store. If the +// partition already exists, the new partition's metadata will overwrite the +// existing metadata, but existing vectors will not be deleted. Vectors in the +// new partition will overwrite existing vectors if child keys collide, but +// otherwise the resulting partition will be a union of the two partitions. +func (psTxn *persistentStoreTxn) insertPartition( + ctx context.Context, partitionKey PartitionKey, partition *Partition, +) error { + b := psTxn.kv.NewBatch() + + key := psTxn.encodePartitionKey(partitionKey) + meta, err := EncodePartitionMetadata(partition.Level(), partition.quantizedSet.GetCentroid()) + if err != nil { + return err + } + b.Put(key, meta) + + codec := psTxn.getCodecForPartitionKey(partitionKey) + childKeys := partition.ChildKeys() + for i := 0; i < partition.quantizedSet.GetCount(); i++ { + // The child key gets appended to 'key' here. + // Cap the metadata key so that the append allocates a new slice for the child key. + key = key[:len(key):len(key)] + k := EncodeChildKey(key, childKeys[i]) + encodedVector, err := codec.encodeVectorFromSet(partition.QuantizedSet(), i) + if err != nil { + return err + } + b.Put(k, encodedVector) + } + + return psTxn.kv.Run(ctx, b) +} + +// SetRootPartition implements the vecstore.Txn interface. +func (psTxn *persistentStoreTxn) SetRootPartition(ctx context.Context, partition *Partition) error { + if err := psTxn.DeletePartition(ctx, RootKey); err != nil { + return err + } + return psTxn.insertPartition(ctx, RootKey, partition) +} + +// InsertPartition implements the vecstore.Txn interface. +func (psTxn *persistentStoreTxn) InsertPartition( + ctx context.Context, partition *Partition, +) (PartitionKey, error) { + instanceID := psTxn.store.db.Context().NodeID.SQLInstanceID() + partitionID := PartitionKey(builtins.GenerateUniqueInt(builtins.ProcessUniqueID(instanceID))) + return partitionID, psTxn.insertPartition(ctx, partitionID, partition) +} + +// DeletePartition implements the vecstore.Txn interface. +func (psTxn *persistentStoreTxn) DeletePartition( + ctx context.Context, partitionKey PartitionKey, +) error { + b := psTxn.kv.NewBatch() + + startKey := psTxn.encodePartitionKey(partitionKey) + endKey := startKey.PrefixEnd() + + b.DelRange(startKey, endKey, false /* returnKeys */) + return psTxn.kv.Run(ctx, b) +} + +func (psTxn *persistentStoreTxn) AddToPartition( + ctx context.Context, partitionKey PartitionKey, vector vector.T, childKey ChildKey, +) (int, error) { + panic("AddToPartition() unimplemented") +} + +func (psTxn *persistentStoreTxn) RemoveFromPartition( + ctx context.Context, partitionKey PartitionKey, childKey ChildKey, +) (int, error) { + panic("RemoveFromPartition() unimplemented") +} + +func (psTxn *persistentStoreTxn) SearchPartitions( + ctx context.Context, + partitionKey []PartitionKey, + queryVector vector.T, + searchSet *SearchSet, + partitionCounts []int, +) (Level, error) { + panic("SearchPartitions() unimplemented") +} + +func (psTxn *persistentStoreTxn) GetFullVectors(ctx context.Context, refs []VectorWithKey) error { + panic("GetFullVectors() unimplemented") +} + +// encodePartitionKey takes a partition key and creates a KV key to read that +// partition's metadata. Vector data can be read by scanning from the metadata to +// the next partition's metadata. +func (psTxn *persistentStoreTxn) encodePartitionKey(partitionKey PartitionKey) roachpb.Key { + keyBuffer := make(roachpb.Key, len(psTxn.store.prefix)+EncodedPartitionKeyLen(partitionKey)) + copy(keyBuffer, psTxn.store.prefix) + keyBuffer = EncodePartitionKey(keyBuffer, partitionKey) + return keyBuffer +} From 6e251fb53c2b4c95994992bfaf5479d359f2bf81 Mon Sep 17 00:00:00 2001 From: Matt White Date: Fri, 27 Dec 2024 13:08:02 -0800 Subject: [PATCH 080/126] vecstore: implement AddTo/RemoveFrom/Search functions Add implementations of AddToPartition, RemoveFromPartition and SearchPartitions. AddToPartition and RemoveFromPartition turn into simple KV operations since each encoded vector is written as a single KV value. SearchPartitions reuses a lot of the code for GetPartition, but without the locking. Epic: CRDB-42943 --- .../vecstore/persistent_store_test.go | 41 +++++++ pkg/sql/vecindex/vecstore/persistent_txn.go | 104 +++++++++++++++++- 2 files changed, 141 insertions(+), 4 deletions(-) diff --git a/pkg/sql/vecindex/vecstore/persistent_store_test.go b/pkg/sql/vecindex/vecstore/persistent_store_test.go index 33ffb821951e..7592df2454d9 100644 --- a/pkg/sql/vecindex/vecstore/persistent_store_test.go +++ b/pkg/sql/vecindex/vecstore/persistent_store_test.go @@ -31,6 +31,7 @@ func TestPersistentStore(t *testing.T) { childKey2 := ChildKey{PartitionKey: 2} childKey10 := ChildKey{PartitionKey: 10} childKey20 := ChildKey{PartitionKey: 20} + childKey30 := ChildKey{PartitionKey: 30} primaryKey200 := ChildKey{PrimaryKey: PrimaryKey{2, 00}} primaryKey300 := ChildKey{PrimaryKey: PrimaryKey{3, 00}} primaryKey400 := ChildKey{PrimaryKey: PrimaryKey{4, 00}} @@ -90,4 +91,44 @@ func TestPersistentStore(t *testing.T) { _, err = txn.GetPartition(ctx, partitionKey) require.Error(t, err) }) + + t.Run("add to root partition", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + emptySet := vector.MakeSet(2) + root := NewPartition(quantizer, quantizer.Quantize(ctx, &emptySet), []ChildKey{}, Level(2)) + err := txn.SetRootPartition(ctx, root) + require.NoError(t, err) + + // Add to root partition. + count, err := txn.AddToPartition(ctx, RootKey, vector.T{1, 2}, childKey10) + require.NoError(t, err) + require.Equal(t, 1, count) + count, err = txn.AddToPartition(ctx, RootKey, vector.T{7, 4}, childKey20) + require.NoError(t, err) + require.Equal(t, 2, count) + count, err = txn.AddToPartition(ctx, RootKey, vector.T{4, 3}, childKey30) + require.NoError(t, err) + require.Equal(t, 3, count) + + // Add duplicate and expect value to be overwritten + count, err = txn.AddToPartition(ctx, RootKey, vector.T{5, 5}, childKey30) + require.NoError(t, err) + require.Equal(t, 3, count) + + // Search root partition. + searchSet := SearchSet{MaxResults: 2} + partitionCounts := []int{0} + level, err := txn.SearchPartitions( + ctx, []PartitionKey{RootKey}, vector.T{1, 1}, &searchSet, partitionCounts) + require.NoError(t, err) + require.Equal(t, Level(2), level) + result1 := SearchResult{QuerySquaredDistance: 1, ErrorBound: 0, CentroidDistance: 2.2361, ParentPartitionKey: 1, ChildKey: childKey10} + result2 := SearchResult{QuerySquaredDistance: 32, ErrorBound: 0, CentroidDistance: 7.0711, ParentPartitionKey: 1, ChildKey: childKey30} + results := searchSet.PopResults() + roundResults(results, 4) + require.Equal(t, SearchResults{result1, result2}, results) + require.Equal(t, 3, partitionCounts[0]) + }) } diff --git a/pkg/sql/vecindex/vecstore/persistent_txn.go b/pkg/sql/vecindex/vecstore/persistent_txn.go index 4326206b666b..fb0e30286535 100644 --- a/pkg/sql/vecindex/vecstore/persistent_txn.go +++ b/pkg/sql/vecindex/vecstore/persistent_txn.go @@ -285,26 +285,122 @@ func (psTxn *persistentStoreTxn) DeletePartition( return psTxn.kv.Run(ctx, b) } +// AddToPartition implements the vecstore.Txn interface. func (psTxn *persistentStoreTxn) AddToPartition( ctx context.Context, partitionKey PartitionKey, vector vector.T, childKey ChildKey, ) (int, error) { - panic("AddToPartition() unimplemented") + // TODO(mw5h): Add to an existing batch instead of starting a new one. + b := psTxn.kv.NewBatch() + + metadataKey := psTxn.encodePartitionKey(partitionKey) + + b.GetForShare(metadataKey, psTxn.lockDurability) + err := psTxn.kv.Run(ctx, b) + if err != nil { + return -1, err + } + if len(b.Results[0].Rows) == 0 { + return -1, ErrPartitionNotFound + } + _, centroid, err := DecodePartitionMetadata(b.Results[0].Rows[0].ValueBytes()) + if err != nil { + return -1, err + } + + // Cap the metadata key so that the append allocates a new slice for the child key. + prefix := metadataKey[:len(metadataKey):len(metadataKey)] + entryKey := EncodeChildKey(prefix, childKey) + + codec := psTxn.getCodecForPartitionKey(partitionKey) + b = psTxn.kv.NewBatch() + encodedVector, err := codec.encodeVector(ctx, vector, centroid) + if err != nil { + return -1, err + } + b.Put(entryKey, encodedVector) + + // This scan is purely for returning partition cardinality. + startKey := psTxn.encodePartitionKey(partitionKey) + endKey := startKey.PrefixEnd() + b.Scan(startKey, endKey) + + if err = psTxn.kv.Run(ctx, b); err != nil { + return -1, err + } + return len(b.Results[1].Rows) - 1, nil } +// RemoveFromPartition implements the vecstore.Txn interface. func (psTxn *persistentStoreTxn) RemoveFromPartition( ctx context.Context, partitionKey PartitionKey, childKey ChildKey, ) (int, error) { - panic("RemoveFromPartition() unimplemented") + b := psTxn.kv.NewBatch() + + // Lock metadata for partition in shared mode to block concurrent fixups. + metadataKey := psTxn.encodePartitionKey(partitionKey) + b.GetForShare(metadataKey, psTxn.lockDurability) + + // Cap the metadata key so that the append allocates a new slice for the child key. + prefix := metadataKey[:len(metadataKey):len(metadataKey)] + entryKey := EncodeChildKey(prefix, childKey) + b.Del(entryKey) + + // Scan to get current cardinality. + startKey := psTxn.encodePartitionKey(partitionKey) + endKey := startKey.PrefixEnd() + b.Scan(startKey, endKey) + + if err := psTxn.kv.Run(ctx, b); err != nil { + return -1, err + } + if len(b.Results[0].Rows) == 0 { + return -1, ErrPartitionNotFound + } + // We ignore key not found for the deleted child. + + return len(b.Results[2].Rows) - 1, nil } +// SearchPartitions implements the vecstore.Txn interface. func (psTxn *persistentStoreTxn) SearchPartitions( ctx context.Context, - partitionKey []PartitionKey, + partitionKeys []PartitionKey, queryVector vector.T, searchSet *SearchSet, partitionCounts []int, ) (Level, error) { - panic("SearchPartitions() unimplemented") + b := psTxn.kv.NewBatch() + + for _, pk := range partitionKeys { + startKey := psTxn.encodePartitionKey(pk) + endKey := startKey.PrefixEnd() + b.Scan(startKey, endKey) + } + + if err := psTxn.kv.Run(ctx, b); err != nil { + return InvalidLevel, err + } + + level := InvalidLevel + codec := psTxn.getCodecForPartitionKey(partitionKeys[0]) + for i, result := range b.Results { + partition, err := psTxn.decodePartition(codec, &result) + if err != nil { + return InvalidLevel, err + } + searchLevel, partitionCount := partition.Search(ctx, partitionKeys[i], queryVector, searchSet) + if i == 0 { + level = searchLevel + } else if level != searchLevel { + // Callers should only search for partitions at the same level. + panic(errors.AssertionFailedf( + "caller already searched a partition at level %d, cannot search at level %d", + level, searchLevel)) + } + partitionCounts[i] = partitionCount + } + + return level, nil } func (psTxn *persistentStoreTxn) GetFullVectors(ctx context.Context, refs []VectorWithKey) error { From 780f5a6c6363fc959086823fa4058529faa3c833 Mon Sep 17 00:00:00 2001 From: Matt White Date: Fri, 27 Dec 2024 13:08:51 -0800 Subject: [PATCH 081/126] vecstore: move vector store implementation tests into a common test Previously, the in-memory and persistent vector stores maintained their own disjoint tests. This patch moves those tests that test common functionality into a single joint test that is referenced from both implementation unit tests. Epic: CRDB-42943 --- pkg/sql/vecindex/vecstore/BUILD.bazel | 1 + .../vecindex/vecstore/in_memory_store_test.go | 177 +--------------- .../vecstore/persistent_store_test.go | 11 + pkg/sql/vecindex/vecstore/store_test.go | 197 ++++++++++++++++++ 4 files changed, 210 insertions(+), 176 deletions(-) create mode 100644 pkg/sql/vecindex/vecstore/store_test.go diff --git a/pkg/sql/vecindex/vecstore/BUILD.bazel b/pkg/sql/vecindex/vecstore/BUILD.bazel index 20c8e09dc071..070948deed5a 100644 --- a/pkg/sql/vecindex/vecstore/BUILD.bazel +++ b/pkg/sql/vecindex/vecstore/BUILD.bazel @@ -70,6 +70,7 @@ go_test( "partition_test.go", "persistent_store_test.go", "search_set_test.go", + "store_test.go", "vecstorepb_test.go", ], embed = [":vecstore"], diff --git a/pkg/sql/vecindex/vecstore/in_memory_store_test.go b/pkg/sql/vecindex/vecstore/in_memory_store_test.go index 75d5a450b9fb..3c8a236957b8 100644 --- a/pkg/sql/vecindex/vecstore/in_memory_store_test.go +++ b/pkg/sql/vecindex/vecstore/in_memory_store_test.go @@ -28,14 +28,6 @@ func TestInMemoryStore(t *testing.T) { ctx := internal.WithWorkspace(context.Background(), &internal.Workspace{}) - childKey2 := ChildKey{PartitionKey: 2} - childKey10 := ChildKey{PartitionKey: 10} - childKey20 := ChildKey{PartitionKey: 20} - childKey30 := ChildKey{PartitionKey: 30} - childKey40 := ChildKey{PartitionKey: 40} - childKey50 := ChildKey{PartitionKey: 50} - childKey60 := ChildKey{PartitionKey: 60} - store := NewInMemoryStore(2, 42) quantizer := quantize.NewUnQuantizer(2) @@ -72,174 +64,7 @@ func TestInMemoryStore(t *testing.T) { }, vectors) }) - t.Run("search empty root partition", func(t *testing.T) { - txn := beginTransaction(ctx, t, store) - defer commitTransaction(ctx, t, store, txn) - - searchSet := SearchSet{MaxResults: 2} - partitionCounts := []int{0} - level, err := txn.SearchPartitions( - ctx, []PartitionKey{RootKey}, vector.T{1, 1}, &searchSet, partitionCounts) - require.NoError(t, err) - require.Equal(t, LeafLevel, level) - require.Nil(t, searchSet.PopResults()) - require.Equal(t, 0, partitionCounts[0]) - }) - - t.Run("add to root partition", func(t *testing.T) { - txn := beginTransaction(ctx, t, store) - defer commitTransaction(ctx, t, store, txn) - - // Add to root partition. - count, err := txn.AddToPartition(ctx, RootKey, vector.T{1, 2}, childKey10) - require.NoError(t, err) - require.Equal(t, 1, count) - count, err = txn.AddToPartition(ctx, RootKey, vector.T{7, 4}, childKey20) - require.NoError(t, err) - require.Equal(t, 2, count) - count, err = txn.AddToPartition(ctx, RootKey, vector.T{4, 3}, childKey30) - require.NoError(t, err) - require.Equal(t, 3, count) - - // Add duplicate and expect value to be overwritten - count, err = txn.AddToPartition(ctx, RootKey, vector.T{5, 5}, childKey30) - require.NoError(t, err) - require.Equal(t, 3, count) - - // Search root partition. - searchSet := SearchSet{MaxResults: 2} - partitionCounts := []int{0} - level, err := txn.SearchPartitions( - ctx, []PartitionKey{RootKey}, vector.T{1, 1}, &searchSet, partitionCounts) - require.NoError(t, err) - require.Equal(t, Level(1), level) - result1 := SearchResult{QuerySquaredDistance: 1, ErrorBound: 0, CentroidDistance: 2.2361, ParentPartitionKey: 1, ChildKey: childKey10} - result2 := SearchResult{QuerySquaredDistance: 32, ErrorBound: 0, CentroidDistance: 7.0711, ParentPartitionKey: 1, ChildKey: childKey30} - results := searchSet.PopResults() - roundResults(results, 4) - require.Equal(t, SearchResults{result1, result2}, results) - require.Equal(t, 3, partitionCounts[0]) - }) - - var root *Partition - t.Run("get root partition", func(t *testing.T) { - txn := beginTransaction(ctx, t, store) - defer commitTransaction(ctx, t, store, txn) - - // Get root partition. - var err error - root, err = txn.GetPartition(ctx, RootKey) - require.NoError(t, err) - require.Equal(t, Level(1), root.Level()) - require.Equal(t, []ChildKey{childKey10, childKey20, childKey30}, root.ChildKeys()) - require.Equal(t, vector.T{0, 0}, root.Centroid()) - - // Get partition centroid + full vectors. - results := []VectorWithKey{ - {Key: ChildKey{PartitionKey: RootKey}}, - {Key: ChildKey{PrimaryKey: PrimaryKey{11}}}, - {Key: ChildKey{PrimaryKey: PrimaryKey{0}}}, - } - err = txn.GetFullVectors(ctx, results) - require.NoError(t, err) - require.Equal(t, vector.T{0, 0}, results[0].Vector) - require.Equal(t, vector.T{100, 200}, results[1].Vector) - require.Nil(t, results[2].Vector) - }) - - t.Run("replace root partition", func(t *testing.T) { - txn := beginTransaction(ctx, t, store) - defer commitTransaction(ctx, t, store, txn) - - // Replace root partition. - _, err := txn.GetPartition(ctx, RootKey) - require.NoError(t, err) - vectors := vector.T{4, 3}.AsSet() - quantizedSet := quantizer.Quantize(ctx, &vectors) - newRoot := NewPartition(quantizer, quantizedSet, []ChildKey{childKey2}, 2) - require.NoError(t, txn.SetRootPartition(ctx, newRoot)) - newRoot, err = txn.GetPartition(ctx, RootKey) - require.NoError(t, err) - require.Equal(t, Level(2), newRoot.Level()) - require.Equal(t, []ChildKey{childKey2}, newRoot.ChildKeys()) - - searchSet := SearchSet{MaxResults: 2} - partitionCounts := []int{0} - level, err := txn.SearchPartitions( - ctx, []PartitionKey{RootKey}, vector.T{2, 2}, &searchSet, partitionCounts) - require.NoError(t, err) - require.Equal(t, Level(2), level) - result3 := SearchResult{QuerySquaredDistance: 5, ErrorBound: 0, CentroidDistance: 0, ParentPartitionKey: 1, ChildKey: childKey2} - require.Equal(t, SearchResults{result3}, searchSet.PopResults()) - require.Equal(t, 1, partitionCounts[0]) - }) - - var partitionKey1 PartitionKey - t.Run("insert another partition and update it", func(t *testing.T) { - txn := beginTransaction(ctx, t, store) - defer commitTransaction(ctx, t, store, txn) - - _, err := txn.GetPartition(ctx, RootKey) - require.NoError(t, err) - partitionKey1, err = txn.InsertPartition(ctx, root) - require.NoError(t, err) - require.Equal(t, PartitionKey(2), partitionKey1) - count, err := txn.RemoveFromPartition(ctx, partitionKey1, childKey20) - require.NoError(t, err) - require.Equal(t, 2, count) - - // Try to remove the same key again. - count, err = txn.RemoveFromPartition(ctx, partitionKey1, childKey20) - require.NoError(t, err) - require.Equal(t, 2, count) - - // Add an alternate element and add duplicate, expecting value to be overwritten. - count, err = txn.AddToPartition(ctx, partitionKey1, vector.T{-1, 0}, childKey40) - require.NoError(t, err) - require.Equal(t, 3, count) - count, err = txn.AddToPartition(ctx, partitionKey1, vector.T{1, 1}, childKey40) - require.NoError(t, err) - require.Equal(t, 3, count) - - searchSet := SearchSet{MaxResults: 2} - partitionCounts := []int{0} - level, err := txn.SearchPartitions( - ctx, []PartitionKey{partitionKey1}, vector.T{1, 1}, &searchSet, partitionCounts) - require.NoError(t, err) - require.Equal(t, Level(1), level) - result4 := SearchResult{QuerySquaredDistance: 0, ErrorBound: 0, CentroidDistance: 1.4142, ParentPartitionKey: 2, ChildKey: childKey40} - result5 := SearchResult{QuerySquaredDistance: 1, ErrorBound: 0, CentroidDistance: 2.2361, ParentPartitionKey: 2, ChildKey: childKey10} - require.Equal(t, SearchResults{result4, result5}, roundResults(searchSet.PopResults(), 4)) - require.Equal(t, 3, partitionCounts[0]) - }) - - t.Run("search multiple partitions at leaf level", func(t *testing.T) { - txn := beginTransaction(ctx, t, store) - defer commitTransaction(ctx, t, store, txn) - - _, err := txn.GetPartition(ctx, RootKey) - require.NoError(t, err) - - vectors := vector.MakeSet(2) - vectors.Add(vector.T{4, -1}) - vectors.Add(vector.T{2, 8}) - quantizedSet := quantizer.Quantize(ctx, &vectors) - partition := NewPartition(quantizer, quantizedSet, []ChildKey{childKey50, childKey60}, LeafLevel) - partitionKey2, err := txn.InsertPartition(ctx, partition) - require.NoError(t, err) - require.Equal(t, PartitionKey(3), partitionKey2) - - searchSet := SearchSet{MaxResults: 2} - partitionCounts := []int{0, 0} - level, err := txn.SearchPartitions( - ctx, []PartitionKey{partitionKey1, partitionKey2}, vector.T{3, 1}, &searchSet, partitionCounts) - require.NoError(t, err) - require.Equal(t, Level(1), level) - result4 := SearchResult{QuerySquaredDistance: 4, ErrorBound: 0, CentroidDistance: 1.41, ParentPartitionKey: 2, ChildKey: childKey40} - result5 := SearchResult{QuerySquaredDistance: 5, ErrorBound: 0, CentroidDistance: 2.24, ParentPartitionKey: 2, ChildKey: childKey10} - require.Equal(t, SearchResults{result4, result5}, roundResults(searchSet.PopResults(), 2)) - require.Equal(t, []int{3, 2}, partitionCounts) - }) + commonStoreTests(ctx, t, store, quantizer) t.Run("delete full vector", func(t *testing.T) { txn := beginTransaction(ctx, t, store) diff --git a/pkg/sql/vecindex/vecstore/persistent_store_test.go b/pkg/sql/vecindex/vecstore/persistent_store_test.go index 7592df2454d9..e9b2a84c2933 100644 --- a/pkg/sql/vecindex/vecstore/persistent_store_test.go +++ b/pkg/sql/vecindex/vecstore/persistent_store_test.go @@ -41,6 +41,17 @@ func TestPersistentStore(t *testing.T) { quantizer := quantize.NewUnQuantizer(2) store := NewPersistentStore(kvDB, quantizer, prefix) + // TODO(mw5h): Figure out where to create the empty root partition. + t.Run("create empty root partition", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + emptyRoot := NewPartition(quantizer, quantizer.Quantize(ctx, &vector.Set{}), []ChildKey{}, LeafLevel) + require.NoError(t, txn.SetRootPartition(ctx, emptyRoot)) + }) + + commonStoreTests(ctx, t, store, quantizer) + t.Run("insert a root partition into the store and read it back", func(t *testing.T) { txn := beginTransaction(ctx, t, store) defer commitTransaction(ctx, t, store, txn) diff --git a/pkg/sql/vecindex/vecstore/store_test.go b/pkg/sql/vecindex/vecstore/store_test.go new file mode 100644 index 000000000000..0c2d6e1f9ebe --- /dev/null +++ b/pkg/sql/vecindex/vecstore/store_test.go @@ -0,0 +1,197 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package vecstore + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/vecindex/quantize" + "github.com/cockroachdb/cockroach/pkg/util/vector" + "github.com/stretchr/testify/require" +) + +func commonStoreTests( + ctx context.Context, t *testing.T, store Store, quantizer quantize.Quantizer, +) { + childKey2 := ChildKey{PartitionKey: 2} + primaryKey100 := ChildKey{PrimaryKey: PrimaryKey{1, 00}} + primaryKey200 := ChildKey{PrimaryKey: PrimaryKey{2, 00}} + primaryKey300 := ChildKey{PrimaryKey: PrimaryKey{3, 00}} + primaryKey400 := ChildKey{PrimaryKey: PrimaryKey{4, 00}} + primaryKey500 := ChildKey{PrimaryKey: PrimaryKey{5, 00}} + primaryKey600 := ChildKey{PrimaryKey: PrimaryKey{6, 00}} + + t.Run("search empty root partition", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + searchSet := SearchSet{MaxResults: 2} + partitionCounts := []int{0} + level, err := txn.SearchPartitions( + ctx, []PartitionKey{RootKey}, vector.T{1, 1}, &searchSet, partitionCounts) + require.NoError(t, err) + require.Equal(t, LeafLevel, level) + require.Nil(t, searchSet.PopResults()) + require.Equal(t, 0, partitionCounts[0]) + }) + + t.Run("add to root partition", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + // Add to root partition. + count, err := txn.AddToPartition(ctx, RootKey, vector.T{1, 2}, primaryKey100) + require.NoError(t, err) + require.Equal(t, 1, count) + count, err = txn.AddToPartition(ctx, RootKey, vector.T{7, 4}, primaryKey200) + require.NoError(t, err) + require.Equal(t, 2, count) + count, err = txn.AddToPartition(ctx, RootKey, vector.T{4, 3}, primaryKey300) + require.NoError(t, err) + require.Equal(t, 3, count) + + // Add duplicate and expect value to be overwritten + count, err = txn.AddToPartition(ctx, RootKey, vector.T{5, 5}, primaryKey300) + require.NoError(t, err) + require.Equal(t, 3, count) + + // Search root partition. + searchSet := SearchSet{MaxResults: 2} + partitionCounts := []int{0} + level, err := txn.SearchPartitions( + ctx, []PartitionKey{RootKey}, vector.T{1, 1}, &searchSet, partitionCounts) + require.NoError(t, err) + require.Equal(t, Level(1), level) + result1 := SearchResult{QuerySquaredDistance: 1, ErrorBound: 0, CentroidDistance: 2.2361, ParentPartitionKey: 1, ChildKey: primaryKey100} + result2 := SearchResult{QuerySquaredDistance: 32, ErrorBound: 0, CentroidDistance: 7.0711, ParentPartitionKey: 1, ChildKey: primaryKey300} + results := searchSet.PopResults() + roundResults(results, 4) + require.Equal(t, SearchResults{result1, result2}, results) + require.Equal(t, 3, partitionCounts[0]) + }) + + var root *Partition + t.Run("get root partition", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + // Get root partition. + var err error + root, err = txn.GetPartition(ctx, RootKey) + require.NoError(t, err) + require.Equal(t, Level(1), root.Level()) + require.Equal(t, []ChildKey{primaryKey100, primaryKey200, primaryKey300}, root.ChildKeys()) + require.Equal(t, vector.T{0, 0}, root.Centroid()) + + // TODO (mw5h): Implement GetFullVectors for PersistentStore. + // Get partition centroid + full vectors. + if _, ok := txn.(*inMemoryTxn); ok { + results := []VectorWithKey{ + {Key: ChildKey{PartitionKey: RootKey}}, + {Key: ChildKey{PrimaryKey: PrimaryKey{11}}}, + {Key: ChildKey{PrimaryKey: PrimaryKey{0}}}, + } + err = txn.GetFullVectors(ctx, results) + require.NoError(t, err) + require.Equal(t, vector.T{0, 0}, results[0].Vector) + require.Equal(t, vector.T{100, 200}, results[1].Vector) + require.Nil(t, results[2].Vector) + } + }) + + t.Run("replace root partition", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + // Replace root partition. + _, err := txn.GetPartition(ctx, RootKey) + require.NoError(t, err) + vectors := vector.T{4, 3}.AsSet() + quantizedSet := quantizer.Quantize(ctx, &vectors) + newRoot := NewPartition(quantizer, quantizedSet, []ChildKey{childKey2}, 2) + require.NoError(t, txn.SetRootPartition(ctx, newRoot)) + newRoot, err = txn.GetPartition(ctx, RootKey) + require.NoError(t, err) + require.Equal(t, Level(2), newRoot.Level()) + require.Equal(t, []ChildKey{childKey2}, newRoot.ChildKeys()) + + searchSet := SearchSet{MaxResults: 2} + partitionCounts := []int{0} + level, err := txn.SearchPartitions( + ctx, []PartitionKey{RootKey}, vector.T{2, 2}, &searchSet, partitionCounts) + require.NoError(t, err) + require.Equal(t, Level(2), level) + result3 := SearchResult{QuerySquaredDistance: 5, ErrorBound: 0, CentroidDistance: 0, ParentPartitionKey: 1, ChildKey: childKey2} + require.Equal(t, SearchResults{result3}, searchSet.PopResults()) + require.Equal(t, 1, partitionCounts[0]) + }) + + var partitionKey1 PartitionKey + t.Run("insert another partition and update it", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + _, err := txn.GetPartition(ctx, RootKey) + require.NoError(t, err) + partitionKey1, err = txn.InsertPartition(ctx, root) + require.NoError(t, err) + count, err := txn.RemoveFromPartition(ctx, partitionKey1, primaryKey200) + require.NoError(t, err) + require.Equal(t, 2, count) + + // Try to remove the same key again. + count, err = txn.RemoveFromPartition(ctx, partitionKey1, primaryKey200) + require.NoError(t, err) + require.Equal(t, 2, count) + + // Add an alternate element and add duplicate, expecting value to be overwritten. + count, err = txn.AddToPartition(ctx, partitionKey1, vector.T{-1, 0}, primaryKey400) + require.NoError(t, err) + require.Equal(t, 3, count) + count, err = txn.AddToPartition(ctx, partitionKey1, vector.T{1, 1}, primaryKey400) + require.NoError(t, err) + require.Equal(t, 3, count) + + searchSet := SearchSet{MaxResults: 2} + partitionCounts := []int{0} + level, err := txn.SearchPartitions( + ctx, []PartitionKey{partitionKey1}, vector.T{1, 1}, &searchSet, partitionCounts) + require.NoError(t, err) + require.Equal(t, Level(1), level) + result4 := SearchResult{QuerySquaredDistance: 0, ErrorBound: 0, CentroidDistance: 1.4142, ParentPartitionKey: partitionKey1, ChildKey: primaryKey400} + result5 := SearchResult{QuerySquaredDistance: 1, ErrorBound: 0, CentroidDistance: 2.2361, ParentPartitionKey: partitionKey1, ChildKey: primaryKey100} + require.Equal(t, SearchResults{result4, result5}, roundResults(searchSet.PopResults(), 4)) + require.Equal(t, 3, partitionCounts[0]) + }) + + t.Run("search multiple partitions at leaf level", func(t *testing.T) { + txn := beginTransaction(ctx, t, store) + defer commitTransaction(ctx, t, store, txn) + + _, err := txn.GetPartition(ctx, RootKey) + require.NoError(t, err) + + vectors := vector.MakeSet(2) + vectors.Add(vector.T{4, -1}) + vectors.Add(vector.T{2, 8}) + quantizedSet := quantizer.Quantize(ctx, &vectors) + partition := NewPartition(quantizer, quantizedSet, []ChildKey{primaryKey500, primaryKey600}, LeafLevel) + partitionKey2, err := txn.InsertPartition(ctx, partition) + require.NoError(t, err) + + searchSet := SearchSet{MaxResults: 2} + partitionCounts := []int{0, 0} + level, err := txn.SearchPartitions( + ctx, []PartitionKey{partitionKey1, partitionKey2}, vector.T{3, 1}, &searchSet, partitionCounts) + require.NoError(t, err) + require.Equal(t, Level(1), level) + result4 := SearchResult{QuerySquaredDistance: 4, ErrorBound: 0, CentroidDistance: 1.41, ParentPartitionKey: partitionKey1, ChildKey: primaryKey400} + result5 := SearchResult{QuerySquaredDistance: 5, ErrorBound: 0, CentroidDistance: 2.24, ParentPartitionKey: partitionKey1, ChildKey: primaryKey100} + require.Equal(t, SearchResults{result4, result5}, roundResults(searchSet.PopResults(), 2)) + require.Equal(t, []int{3, 2}, partitionCounts) + }) +} From 3c7e2358c9be800742ac2ec0d8083622eba8d3dd Mon Sep 17 00:00:00 2001 From: Andy Yang Date: Thu, 9 Jan 2025 17:01:23 -0500 Subject: [PATCH 082/126] changefeedccl: rename changefeed.frontier_checkpoint_max_bytes setting This patch renames `changefeed.frontier_checkpoint_max_bytes` to `changefeed.span_checkpoint.max_bytes` for consistency. Release note: None --- pkg/ccl/changefeedccl/alter_changefeed_test.go | 4 ++-- pkg/ccl/changefeedccl/changefeed_processors.go | 2 +- pkg/ccl/changefeedccl/changefeed_stmt.go | 2 +- pkg/ccl/changefeedccl/changefeed_test.go | 8 ++++---- pkg/ccl/changefeedccl/changefeedbase/settings.go | 9 +++++---- pkg/ccl/changefeedccl/checkpoint/checkpoint_test.go | 2 +- 6 files changed, 14 insertions(+), 13 deletions(-) diff --git a/pkg/ccl/changefeedccl/alter_changefeed_test.go b/pkg/ccl/changefeedccl/alter_changefeed_test.go index 3e33dfbf7567..bd93464d5166 100644 --- a/pkg/ccl/changefeedccl/alter_changefeed_test.go +++ b/pkg/ccl/changefeedccl/alter_changefeed_test.go @@ -1223,7 +1223,7 @@ func TestAlterChangefeedAddTargetsDuringSchemaChangeError(t *testing.T) { // setting 1 nanosecond is the smallest possible value. changefeedbase.FrontierCheckpointFrequency.Override( context.Background(), &s.Server.ClusterSettings().SV, 1*time.Nanosecond) - changefeedbase.FrontierCheckpointMaxBytes.Override( + changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, maxCheckpointSize) // Note the tableSpan to avoid resolved events that leave no gaps @@ -1387,7 +1387,7 @@ func TestAlterChangefeedAddTargetsDuringBackfill(t *testing.T) { // Checkpoint progress frequently, and set the checkpoint size limit. changefeedbase.FrontierCheckpointFrequency.Override( context.Background(), &s.Server.ClusterSettings().SV, 1) - changefeedbase.FrontierCheckpointMaxBytes.Override( + changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, maxCheckpointSize) registry := s.Server.JobRegistry().(*jobs.Registry) diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index 0cc941df7383..c0848b046b92 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -1675,7 +1675,7 @@ func (cf *changeFrontier) maybeCheckpointJob( // If the highwater has moved an empty checkpoint will be saved var checkpoint jobspb.ChangefeedProgress_Checkpoint if updateCheckpoint { - maxBytes := changefeedbase.FrontierCheckpointMaxBytes.Get(&cf.FlowCtx.Cfg.Settings.SV) + maxBytes := changefeedbase.SpanCheckpointMaxBytes.Get(&cf.FlowCtx.Cfg.Settings.SV) checkpoint = cf.frontier.MakeCheckpoint(maxBytes) } diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index 7ea75cbe4aa1..958bf29d2aa2 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -1499,7 +1499,7 @@ func reconcileJobStateWithLocalState( } } - maxBytes := changefeedbase.FrontierCheckpointMaxBytes.Get(&execCfg.Settings.SV) + maxBytes := changefeedbase.SpanCheckpointMaxBytes.Get(&execCfg.Settings.SV) checkpoint := checkpoint.Make(sf.Frontier(), func(forEachSpan span.Operation) { for _, fs := range localState.aggregatorFrontier { forEachSpan(fs.Span, fs.Timestamp) diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index e9ebd82b77cd..9968b5798890 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -2338,7 +2338,7 @@ func TestChangefeedLaggingSpanCheckpointing(t *testing.T) { // reduce the lag threshold to allow lag checkpointing to trigger changefeedbase.FrontierCheckpointFrequency.Override( context.Background(), &s.ClusterSettings().SV, 10*time.Millisecond) - changefeedbase.FrontierCheckpointMaxBytes.Override( + changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.ClusterSettings().SV, 100<<20) changefeedbase.FrontierHighwaterLagCheckpointThreshold.Override( context.Background(), &s.ClusterSettings().SV, 10*time.Millisecond) @@ -2524,7 +2524,7 @@ func TestChangefeedSchemaChangeBackfillCheckpoint(t *testing.T) { // Checkpoint progress frequently, and set the checkpoint size limit. changefeedbase.FrontierCheckpointFrequency.Override( context.Background(), &s.Server.ClusterSettings().SV, 1) - changefeedbase.FrontierCheckpointMaxBytes.Override( + changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, maxCheckpointSize) var tableSpan roachpb.Span @@ -7343,7 +7343,7 @@ func TestChangefeedBackfillCheckpoint(t *testing.T) { // Checkpoint progress frequently, and set the checkpoint size limit. changefeedbase.FrontierCheckpointFrequency.Override( context.Background(), &s.Server.ClusterSettings().SV, 1) - changefeedbase.FrontierCheckpointMaxBytes.Override( + changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, maxCheckpointSize) registry := s.Server.JobRegistry().(*jobs.Registry) @@ -7494,7 +7494,7 @@ func TestCoreChangefeedBackfillScanCheckpoint(t *testing.T) { } changefeedbase.FrontierCheckpointFrequency.Override( context.Background(), &s.Server.ClusterSettings().SV, 1) - changefeedbase.FrontierCheckpointMaxBytes.Override( + changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, 100<<20) emittedCount := 0 diff --git a/pkg/ccl/changefeedccl/changefeedbase/settings.go b/pkg/ccl/changefeedccl/changefeedbase/settings.go index e327c13c9757..c1ae7aec921e 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/settings.go +++ b/pkg/ccl/changefeedccl/changefeedbase/settings.go @@ -87,8 +87,8 @@ var FrontierHighwaterLagCheckpointThreshold = settings.RegisterDurationSetting( settings.NonNegativeDuration, settings.WithPublic) -// FrontierCheckpointMaxBytes controls the maximum number of key bytes that will be added -// to the checkpoint record. +// SpanCheckpointMaxBytes controls the maximum number of key bytes that will be added +// to a span-level checkpoint record. // Checkpoint record could be fairly large. // Assume we have a 10T table, and a 1/2G max range size: 20K spans. // Span frontier merges adjacent spans, so worst case we have 10K spans. @@ -101,11 +101,12 @@ var FrontierHighwaterLagCheckpointThreshold = settings.RegisterDurationSetting( // // Therefore, we should write at most 6 MB of checkpoint/hour; OR, based on the default // FrontierCheckpointFrequency setting, 1 MB per checkpoint. -var FrontierCheckpointMaxBytes = settings.RegisterByteSizeSetting( +var SpanCheckpointMaxBytes = settings.RegisterByteSizeSetting( settings.ApplicationLevel, "changefeed.frontier_checkpoint_max_bytes", - "controls the maximum size of the checkpoint as a total size of key bytes", + "the maximum size of a changefeed span-level checkpoint as measured by the total size of key bytes", 1<<20, // 1 MiB + settings.WithName("changefeed.span_checkpoint.max_bytes"), ) // ScanRequestLimit is the number of Scan requests that can run at once. diff --git a/pkg/ccl/changefeedccl/checkpoint/checkpoint_test.go b/pkg/ccl/changefeedccl/checkpoint/checkpoint_test.go index aa08933ee509..585da42e1d38 100644 --- a/pkg/ccl/changefeedccl/checkpoint/checkpoint_test.go +++ b/pkg/ccl/changefeedccl/checkpoint/checkpoint_test.go @@ -151,7 +151,7 @@ func TestCheckpointCatchupTime(t *testing.T) { defer log.Scope(t).Close(t) const numSpans = 100 - maxBytes := changefeedbase.FrontierCheckpointMaxBytes.Default() + maxBytes := changefeedbase.SpanCheckpointMaxBytes.Default() hwm := hlc.Timestamp{} rng, _ := randutil.NewTestRand() From 428227821643c53ae8dfbb8976f21de4eaae950e Mon Sep 17 00:00:00 2001 From: Chandra Thumuluru Date: Mon, 13 Jan 2025 14:13:22 +0000 Subject: [PATCH 083/126] server: simple query test with DPRC enabled/disabled. Epic: None Release note: None --- pkg/server/BUILD.bazel | 1 + pkg/server/server_drpc_test.go | 57 ++++++++++++++++++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 pkg/server/server_drpc_test.go diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index a9879a6b443b..2ed0eb3f88de 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -463,6 +463,7 @@ go_test( "purge_auth_session_test.go", "server_controller_http_test.go", "server_controller_test.go", + "server_drpc_test.go", "server_http_test.go", "server_import_ts_test.go", "server_internal_executor_factory_test.go", diff --git a/pkg/server/server_drpc_test.go b/pkg/server/server_drpc_test.go new file mode 100644 index 000000000000..fe7f44fbccff --- /dev/null +++ b/pkg/server/server_drpc_test.go @@ -0,0 +1,57 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package server + +import ( + "context" + "strconv" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/envutil" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestSelectQuery(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Run the test with both DRPC enabled and disabled. + testutils.RunTrueAndFalse(t, "enableDRPC", func(t *testing.T, enableDRPC bool) { + clear := envutil.TestSetEnv(t, "COCKROACH_EXPERIMENTAL_DRPC_ENABLED", strconv.FormatBool(enableDRPC)) + defer clear() + + ctx := context.Background() + tc := serverutils.StartCluster(t, 3, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Insecure: true, + }, + }) + defer tc.Stopper().Stop(ctx) + + db := tc.ServerConn(1) + defer db.Close() + + ctx, cancel := context.WithTimeout(ctx, time.Second) + defer cancel() + + rows, err := db.QueryContext(ctx, "SELECT count(*) FROM system.tenants") + require.NoError(t, err) + defer rows.Close() + + for rows.Next() { + var count int + require.NoError(t, rows.Scan(&count)) + require.Equal(t, 1, count) + } + require.NoError(t, rows.Close()) + }) +} From b0ea57cf550d1cb039d34221f9613ccbfb23da28 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 14 Jan 2025 09:45:56 +0100 Subject: [PATCH 084/126] server: fix drpc end-to-end test It wasn't actually exercising drpc. This is because by the time the env var is overridden, the previous value has already been picked up by the cluster setting. Changed the test to override the setting instead. Also, removed the drpc-off run of the test: all other tests already run with drpc off, so and the test takes a second or two, so not worth it. Instead, added a secure flavor. This secure flavor promptly failed (once drpc was actually enabled) because it needed the same hack as the original prototype[1], which I then proceeded to add. [1]: https://github.com/cockroachdb/cockroach/pull/136794/files#diff-fc54146a422d3d59215aae7d10ecb55771db57f0f49b0b0fa34cd64358979ca0R293 --- pkg/rpc/drpc.go | 3 +++ pkg/server/server_drpc_test.go | 31 ++++++++++++++++--------------- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/pkg/rpc/drpc.go b/pkg/rpc/drpc.go index 4b8e342d2fbc..c67f1d6642b5 100644 --- a/pkg/rpc/drpc.go +++ b/pkg/rpc/drpc.go @@ -121,6 +121,9 @@ func dialDRPC(rpcCtx *Context) func(ctx context.Context, target string) (drpcpoo return nil, err } tlsConn := tls.Client(netConn, tlsConfig) + // TODO(server): remove this hack which is necessary at least in + // testing to get TestDRPCSelectQuery to pass. + tlsConfig.InsecureSkipVerify = true conn = drpcconn.NewWithOptions(tlsConn, opts) } diff --git a/pkg/server/server_drpc_test.go b/pkg/server/server_drpc_test.go index fe7f44fbccff..2873169a103a 100644 --- a/pkg/server/server_drpc_test.go +++ b/pkg/server/server_drpc_test.go @@ -7,42 +7,43 @@ package server import ( "context" - "strconv" + "math/rand" "testing" - "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/stretchr/testify/require" ) -func TestSelectQuery(t *testing.T) { +func TestDRPCSelectQuery(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - // Run the test with both DRPC enabled and disabled. - testutils.RunTrueAndFalse(t, "enableDRPC", func(t *testing.T, enableDRPC bool) { - clear := envutil.TestSetEnv(t, "COCKROACH_EXPERIMENTAL_DRPC_ENABLED", strconv.FormatBool(enableDRPC)) - defer clear() + testutils.RunTrueAndFalse(t, "insecure", func(t *testing.T, insecure bool) { + ctx, cancel := context.WithTimeout(context.Background(), testutils.SucceedsSoonDuration()) + defer cancel() + + st := cluster.MakeTestingClusterSettings() + rpc.ExperimentalDRPCEnabled.Override(ctx, &st.SV, true) - ctx := context.Background() tc := serverutils.StartCluster(t, 3, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - Insecure: true, + Settings: st, + Insecure: insecure, }, }) defer tc.Stopper().Stop(ctx) - db := tc.ServerConn(1) + idx := rand.Intn(tc.NumServers()) + t.Logf("querying from node %d", idx+1) + db := tc.ServerConn(idx) defer db.Close() - ctx, cancel := context.WithTimeout(ctx, time.Second) - defer cancel() - rows, err := db.QueryContext(ctx, "SELECT count(*) FROM system.tenants") require.NoError(t, err) defer rows.Close() @@ -52,6 +53,6 @@ func TestSelectQuery(t *testing.T) { require.NoError(t, rows.Scan(&count)) require.Equal(t, 1, count) } - require.NoError(t, rows.Close()) + require.NoError(t, rows.Err()) }) } From 6673a74e0e1a4f1ca5244391da6f23b0bf9d299c Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 10 Jan 2025 17:10:09 +0100 Subject: [PATCH 085/126] scripts: add trigger-pr-roachtest.sh This is useful for PRs that make changes to a roachtest. The added script makes it extremely simply to trigger a roachtest run in our CI infrastructure, from the given PR and test regexp. Doing this was always possible from the UI, but it always takes a few minutes to figure out and I assume many just didn't know about it. Let's make this more commonplace to save time; switching to a gceworker to try out the change manually is time-consuming and the resulting artifacts aren't automatically shared with collaborators. Epic: none Release note: None --- scripts/trigger-pr-roachtest.sh | 57 +++++++++++++++++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100755 scripts/trigger-pr-roachtest.sh diff --git a/scripts/trigger-pr-roachtest.sh b/scripts/trigger-pr-roachtest.sh new file mode 100755 index 000000000000..f2699ea0ca26 --- /dev/null +++ b/scripts/trigger-pr-roachtest.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash + +# Copyright 2016 The Cockroach Authors. +# +# Use of this software is governed by the CockroachDB Software License +# included in the /LICENSE file. + +set -euo pipefail + +token=${TEAMCITY_TOKEN-} +if [ -z $token ]; then + cat < Date: Mon, 23 Dec 2024 14:17:06 +0100 Subject: [PATCH 086/126] roachprod: disable update check when opted out or update unavailable The update check takes >800ms where I am. `roachprod` is slow enough as is, so allow folks to opt out and also opt out automatically in cases where `roachprod update` doesn't work in the first place (osx). Epic: none Release note: None --- pkg/cmd/roachprod/cli/commands.go | 9 +++++++-- pkg/cmd/roachprod/cli/handlers.go | 19 +++++++++++-------- 2 files changed, 18 insertions(+), 10 deletions(-) diff --git a/pkg/cmd/roachprod/cli/commands.go b/pkg/cmd/roachprod/cli/commands.go index e0dbf5d683cb..7082e5f9166b 100644 --- a/pkg/cmd/roachprod/cli/commands.go +++ b/pkg/cmd/roachprod/cli/commands.go @@ -1935,6 +1935,11 @@ func buildSnapshotApplyCmd() *cobra.Command { } } +func roachprodUpdateSupported(goos, goarch string) bool { + // We only have prebuilt binaries for Linux. See #120750. + return goos == "linux" +} + func (cr *commandRegistry) buildUpdateCmd() *cobra.Command { updateCmd := &cobra.Command{ Use: "update", @@ -1944,8 +1949,8 @@ func (cr *commandRegistry) buildUpdateCmd() *cobra.Command { " and can be restored via `roachprod update --revert`.", Run: wrap(func(cmd *cobra.Command, args []string) error { // We only have prebuilt binaries for Linux. See #120750. - if runtime.GOOS != "linux" { - return errors.New("this command is only available on Linux at this time") + if !roachprodUpdateSupported(runtime.GOOS, runtime.GOARCH) { + return errors.Errorf("this command is not available on %s/%s at this time", runtime.GOOS, runtime.GOARCH) } currentBinary, err := os.Executable() diff --git a/pkg/cmd/roachprod/cli/handlers.go b/pkg/cmd/roachprod/cli/handlers.go index 152e00fe00ff..89826df61d7a 100644 --- a/pkg/cmd/roachprod/cli/handlers.go +++ b/pkg/cmd/roachprod/cli/handlers.go @@ -8,6 +8,7 @@ import ( "fmt" "os" "os/user" + "runtime" "github.com/cockroachdb/cockroach/pkg/roachprod" "github.com/cockroachdb/cockroach/pkg/roachprod/config" @@ -76,15 +77,17 @@ func Initialize(rootCmd *cobra.Command) { fmt.Printf("problem loading clusters: %s\n", err) } - updateTime, sha, err := CheckLatest(roachprodUpdateBranch, roachprodUpdateOS, roachprodUpdateArch) - if err != nil { - fmt.Fprintf(os.Stderr, "WARN: failed to check if a more recent 'roachprod' binary exists: %s\n", err) - } else { - age, err := TimeSinceUpdate(updateTime) + if roachprodUpdateSupported(runtime.GOOS, runtime.GOARCH) && os.Getenv("ROACHPROD_DISABLE_UPDATE_CHECK") != "true" { + updateTime, sha, err := CheckLatest(roachprodUpdateBranch, roachprodUpdateOS, roachprodUpdateArch) if err != nil { - fmt.Fprintf(os.Stderr, "WARN: unable to check mtime of 'roachprod' binary: %s\n", err) - } else if age.Hours() >= 14*24 { - fmt.Fprintf(os.Stderr, "WARN: roachprod binary is >= 2 weeks old (%s); latest sha: %q\nWARN: Consider updating the binary: `roachprod update`\n\n", age, sha) + fmt.Fprintf(os.Stderr, "WARN: failed to check if a more recent 'roachprod' binary exists: %s\n", err) + } else { + age, err := TimeSinceUpdate(updateTime) + if err != nil { + fmt.Fprintf(os.Stderr, "WARN: unable to check mtime of 'roachprod' binary: %s\n", err) + } else if age.Hours() >= 14*24 { + fmt.Fprintf(os.Stderr, "WARN: roachprod binary is >= 2 weeks old (%s); latest sha: %q\nWARN: Consider updating the binary: `roachprod update`\n\n", age, sha) + } } } } From ffd28bfa24cb4da377699361dc8ca8630854d6d3 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 23 Dec 2024 15:11:31 +0100 Subject: [PATCH 087/126] roachprod: allow opting out of awscli version check It takes like 400ms. --- pkg/roachprod/vm/aws/aws.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/roachprod/vm/aws/aws.go b/pkg/roachprod/vm/aws/aws.go index 78895822d5f8..b8a139ee56fb 100644 --- a/pkg/roachprod/vm/aws/aws.go +++ b/pkg/roachprod/vm/aws/aws.go @@ -62,6 +62,10 @@ func Init() error { providerInstance.IAMProfile = "roachprod-testing" haveRequiredVersion := func() bool { + // `aws --version` takes around 400ms on my machine. + if os.Getenv("ROACHPROD_SKIP_AWSCLI_CHECK") == "true" { + return true + } cmd := exec.Command("aws", "--version") output, err := cmd.Output() if err != nil { From 7f30eeb60233f9293c9b89462859f56d289275f8 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 23 Dec 2024 15:12:32 +0100 Subject: [PATCH 088/126] roachprod: allow opting out some providers Azure in particular takes a long time - sometimes well north of 1s. --- pkg/roachprod/BUILD.bazel | 1 + pkg/roachprod/roachprod.go | 70 ++++++++++++++++++++++++++------------ 2 files changed, 49 insertions(+), 22 deletions(-) diff --git a/pkg/roachprod/BUILD.bazel b/pkg/roachprod/BUILD.bazel index 203bbdf08710..74b9b9391b4b 100644 --- a/pkg/roachprod/BUILD.bazel +++ b/pkg/roachprod/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//pkg/roachprod/vm", "//pkg/roachprod/vm/aws", "//pkg/roachprod/vm/azure", + "//pkg/roachprod/vm/flagstub", "//pkg/roachprod/vm/gce", "//pkg/roachprod/vm/local", "//pkg/server/debug/replay", diff --git a/pkg/roachprod/roachprod.go b/pkg/roachprod/roachprod.go index 3fea70042582..da4b96a9cc09 100644 --- a/pkg/roachprod/roachprod.go +++ b/pkg/roachprod/roachprod.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/roachprod/vm/aws" "github.com/cockroachdb/cockroach/pkg/roachprod/vm/azure" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm/flagstub" "github.com/cockroachdb/cockroach/pkg/roachprod/vm/gce" "github.com/cockroachdb/cockroach/pkg/roachprod/vm/local" "github.com/cockroachdb/cockroach/pkg/server/debug/replay" @@ -1844,33 +1845,58 @@ func StageURL( return urls, nil } +var disabledProviders = func() map[string]struct{} { + disabled := make(map[string]struct{}) + for _, p := range strings.Split(os.Getenv("ROACHPROD_DISABLED_PROVIDERS"), ",") { + disabled[strings.TrimSpace(strings.ToLower(p))] = struct{}{} + } + return disabled +}() + // InitProviders initializes providers and returns a map that indicates // if a provider is active or inactive. func InitProviders() map[string]string { providersState := make(map[string]string) - if err := aws.Init(); err != nil { - providersState[aws.ProviderName] = "Inactive - " + err.Error() - } else { - providersState[aws.ProviderName] = "Active" - } - - if err := gce.Init(); err != nil { - providersState[gce.ProviderName] = "Inactive - " + err.Error() - } else { - providersState[gce.ProviderName] = "Active" - } - - if err := azure.Init(); err != nil { - providersState[azure.ProviderName] = "Inactive - " + err.Error() - } else { - providersState[azure.ProviderName] = "Active" - } - - if err := local.Init(localVMStorage{}); err != nil { - providersState[local.ProviderName] = "Inactive - " + err.Error() - } else { - providersState[local.ProviderName] = "Active" + for _, prov := range []struct { + name string + init func() error + empty vm.Provider + }{ + { + name: aws.ProviderName, + init: aws.Init, + empty: &aws.Provider{}, + }, + { + name: gce.ProviderName, + init: gce.Init, + empty: &gce.Provider{}, + }, + { + name: azure.ProviderName, + init: azure.Init, + empty: &azure.Provider{}, + }, + { + name: local.ProviderName, + init: func() error { + return local.Init(localVMStorage{}) + }, + empty: &local.Provider{}, + }, + } { + if _, dis := disabledProviders[prov.name]; dis { + reason := "disabled via ROACHPROD_DISABLED_PROVIDERS" + providersState[prov.name] = "Inactive - " + reason + // We need an empty provider that emits errors or we'll + // crash as roachprod expects all providers to be present. + vm.Providers[prov.name] = flagstub.New(prov.empty, reason) + } else if err := prov.init(); err != nil { + providersState[prov.name] = "Inactive - " + err.Error() + } else { + providersState[prov.name] = "Active" + } } return providersState From ec14fc4d62dc72c61f2723d2978894fa99043186 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 10 Jan 2025 10:51:19 +0100 Subject: [PATCH 089/126] roachtest: improve artifacts' tsdump-run.sh Use a local cluster and avoid the need for any manual setup, while also allowing user to specify the version to host the tsdump with. Epic: none Release note: None --- pkg/cmd/roachtest/BUILD.bazel | 1 + pkg/cmd/roachtest/cluster.go | 9 ++++---- pkg/cmd/roachtest/tsdump-run.sh | 39 +++++++++++++++++++++++++++++++++ 3 files changed, 45 insertions(+), 4 deletions(-) create mode 100644 pkg/cmd/roachtest/tsdump-run.sh diff --git a/pkg/cmd/roachtest/BUILD.bazel b/pkg/cmd/roachtest/BUILD.bazel index 1c1d317e1312..5b7aaa274e57 100644 --- a/pkg/cmd/roachtest/BUILD.bazel +++ b/pkg/cmd/roachtest/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "work_pool.go", "zip_util.go", ], + embedsrcs = ["tsdump-run.sh"], importpath = "github.com/cockroachdb/cockroach/pkg/cmd/roachtest", visibility = ["//visibility:private"], deps = [ diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 2ba883b15e26..be5661403c65 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -9,6 +9,7 @@ import ( "bytes" "context" gosql "database/sql" + _ "embed" "encoding/json" "fmt" "io" @@ -58,6 +59,9 @@ func init() { _ = roachprod.InitProviders() } +//go:embed tsdump-run.sh +var tsdumpRunSh string + var ( // maps cpuArch to the corresponding crdb binary's absolute path cockroach = make(map[vm.CPUArch]string) @@ -1358,10 +1362,7 @@ func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, l *logger.Logger) if err := os.WriteFile(tsDumpGob+".yaml", buf.Bytes(), 0644); err != nil { return err } - return os.WriteFile(tsDumpGob+"-run.sh", []byte(`#!/usr/bin/env bash - -COCKROACH_DEBUG_TS_IMPORT_FILE=tsdump.gob cockroach start-single-node --insecure $* -`), 0755) + return os.WriteFile(tsDumpGob+"-run.sh", []byte(tsdumpRunSh), 0755) }) } diff --git a/pkg/cmd/roachtest/tsdump-run.sh b/pkg/cmd/roachtest/tsdump-run.sh new file mode 100644 index 000000000000..3485a5d6d020 --- /dev/null +++ b/pkg/cmd/roachtest/tsdump-run.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash + +# Copyright 2016 The Cockroach Authors. +# +# Use of this software is governed by the CockroachDB Software License +# included in the /LICENSE file. + +set -euo pipefail + +c=local-tsdump + +if [ $# -eq 0 ]; then + cat < /dev/null || true +roachprod create -n 1 $c + +if [ -f "$1" ]; then + roachprod put $c "$1" ./cockroach +else + roachprod stage $c "$@" +fi + +roachprod start --insecure --env COCKROACH_DEBUG_TS_IMPORT_FILE="${PWD}/tsdump.gob" $c + +roachprod adminurl --insecure $c:1 + +trap "roachprod destroy $c" EXIT +echo "Running; hit CTRL-C to destroy the cluster" +while true; do sleep 86400; done From f7a3d5d57eb5c4350ec7045df8c7b1e22f64fc0c Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 14 Jan 2025 10:27:26 -0500 Subject: [PATCH 090/126] sql/logictest: disable column family mutations in some cases Random column family mutations are now disabled for `CREATE TABLE` statements with unique, hash-sharded indexes. This prevents the AST from being reserialized with a `UNIQUE` constraint with invalid options, instead of the original `UNIQUE INDEX`. See #65929 and #107398. Release note: None --- pkg/sql/logictest/logic.go | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 787f984b4b1d..89ca74238dee 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -3495,17 +3495,28 @@ func (t *logicTest) unexpectedError(sql string, pos string, err error) (bool, er return false, fmt.Errorf("%s: %s\nexpected success, but found\n%s", pos, sql, formatErr(err)) } +var uniqueHashPattern = regexp.MustCompile(`UNIQUE.*USING\s+HASH`) + func (t *logicTest) execStatement(stmt logicStatement) (bool, error) { db := t.db t.noticeBuffer = nil if *showSQL { t.outf("%s;", stmt.sql) } - execSQL, changed := randgen.ApplyString(t.rng, stmt.sql, randgen.ColumnFamilyMutator) - if changed { - log.Infof(context.Background(), "Rewrote test statement:\n%s", execSQL) - if *showSQL { - t.outf("rewrote:\n%s\n", execSQL) + execSQL := stmt.sql + // TODO(#65929, #107398): Don't mutate column families for CREATE TABLE + // statements with unique, hash-sharded indexes. The altered AST will be + // reserialized with a UNIQUE constraint, not a UNIQUE INDEX, which may not + // be parsable because constraints do not support all the options that + // indexes do. + if !uniqueHashPattern.MatchString(stmt.sql) { + var changed bool + execSQL, changed = randgen.ApplyString(t.rng, execSQL, randgen.ColumnFamilyMutator) + if changed { + log.Infof(context.Background(), "Rewrote test statement:\n%s", execSQL) + if *showSQL { + t.outf("rewrote:\n%s\n", execSQL) + } } } @@ -3535,8 +3546,6 @@ func (t *logicTest) execStatement(stmt logicStatement) (bool, error) { return t.finishExecStatement(stmt, execSQL, res, err) } -var uniqueHashPattern = regexp.MustCompile(`UNIQUE.*USING\s+HASH`) - func (t *logicTest) finishExecStatement( stmt logicStatement, execSQL string, res gosql.Result, err error, ) (bool, error) { From 3fb65dd928cb7939a6c91bbcb67606e1f4b295b8 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Tue, 14 Jan 2025 08:43:41 -0500 Subject: [PATCH 091/126] crosscluster/logical: replace description with command in SHOW LDR JOBS Fixes #136528 Release note: none --- .../create_logical_replication_stmt.go | 1 + .../logical/logical_replication_job_test.go | 29 ++++++++++--------- pkg/jobs/jobspb/jobs.proto | 4 ++- .../delegate/show_logical_replication_jobs.go | 2 +- 4 files changed, 21 insertions(+), 15 deletions(-) diff --git a/pkg/crosscluster/logical/create_logical_replication_stmt.go b/pkg/crosscluster/logical/create_logical_replication_stmt.go index 43c8ded1542a..24cc9926a868 100644 --- a/pkg/crosscluster/logical/create_logical_replication_stmt.go +++ b/pkg/crosscluster/logical/create_logical_replication_stmt.go @@ -289,6 +289,7 @@ func createLogicalReplicationStreamPlanHook( CreateTable: stmt.CreateTable, ReverseStreamCommand: reverseStreamCmd, ParentID: int64(options.ParentID), + Command: stmt.String(), }, Progress: progress, } diff --git a/pkg/crosscluster/logical/logical_replication_job_test.go b/pkg/crosscluster/logical/logical_replication_job_test.go index 6de3ed461f2b..4556a6ae9ff7 100644 --- a/pkg/crosscluster/logical/logical_replication_job_test.go +++ b/pkg/crosscluster/logical/logical_replication_job_test.go @@ -1867,21 +1867,18 @@ func TestShowLogicalReplicationJobs(t *testing.T) { serverutils.UserPassword(username.RootUser, "password")) defer cleanupB() - redactedDbAURL := strings.Replace(dbAURL.String(), "password", `redacted`, 1) - redactedDbBURL := strings.Replace(dbBURL.String(), "password", `redacted`, 1) - - redactedJobADescription := fmt.Sprintf("LOGICAL REPLICATION STREAM into a.public.tab from %s", redactedDbBURL) - redactedJobBDescription := fmt.Sprintf("LOGICAL REPLICATION STREAM into b.public.tab from %s", redactedDbAURL) - var ( jobAID jobspb.JobID jobBID jobspb.JobID ) + + cmdA := "CREATE LOGICAL REPLICATION STREAM FROM TABLE tab ON $1 INTO TABLE tab" + cmdB := "CREATE LOGICAL REPLICATION STREAM FROM TABLE tab ON $1 INTO TABLE tab WITH OPTIONS (DEFAULT FUNCTION = 'dlq')" dbA.QueryRow(t, - "CREATE LOGICAL REPLICATION STREAM FROM TABLE tab on $1 INTO TABLE tab", + cmdA, dbBURL.String()).Scan(&jobAID) dbB.QueryRow(t, - "CREATE LOGICAL REPLICATION STREAM FROM TABLE tab on $1 INTO TABLE tab WITH DEFAULT FUNCTION = 'dlq'", + cmdB, dbAURL.String()).Scan(&jobBID) now := s.Clock().Now() @@ -1906,7 +1903,7 @@ func TestShowLogicalReplicationJobs(t *testing.T) { replicatedTime time.Time replicationStartTime time.Time conflictResolutionType string - description string + command string ) showRows := dbA.Query(t, "SELECT * FROM [SHOW LOGICAL REPLICATION JOBS] ORDER BY job_id") @@ -1949,7 +1946,7 @@ func TestShowLogicalReplicationJobs(t *testing.T) { &replicatedTime, &replicationStartTime, &conflictResolutionType, - &description) + &command) require.NoError(t, err) expectedJobID := jobIDs[rowIdx] @@ -1962,15 +1959,21 @@ func TestShowLogicalReplicationJobs(t *testing.T) { expectedJobDescription := payload.Description - // Verify that URL is redacted in job descriptions + // Verify that URL is redacted in job descriptions. Note these do not appear + // in SHOW LDR JOBS, but do in the db console description. + redactedDbAURL := strings.Replace(dbAURL.String(), "password", `redacted`, 1) + redactedDbBURL := strings.Replace(dbBURL.String(), "password", `redacted`, 1) + redactedJobADescription := fmt.Sprintf("LOGICAL REPLICATION STREAM into a.public.tab from %s", redactedDbBURL) + redactedJobBDescription := fmt.Sprintf("LOGICAL REPLICATION STREAM into b.public.tab from %s", redactedDbAURL) if jobID == jobAID { require.Equal(t, redactedJobADescription, expectedJobDescription) + require.Equal(t, cmdA, command) + } else if jobID == jobBID { require.Equal(t, redactedJobBDescription, expectedJobDescription) + require.Equal(t, cmdB, command) } - require.Equal(t, expectedJobDescription, description) - rowIdx++ } require.Equal(t, 2, rowIdx) diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index da90ffb04b56..9ae417920c00 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -290,7 +290,9 @@ message LogicalReplicationDetails { // command. int64 parent_id = 15 [(gogoproto.customname) = "ParentID"]; - // Next ID: 16. + string command = 16; + + // Next ID: 17. } message LogicalReplicationProgress { diff --git a/pkg/sql/delegate/show_logical_replication_jobs.go b/pkg/sql/delegate/show_logical_replication_jobs.go index 18b7f07c0764..433ba79fcb21 100644 --- a/pkg/sql/delegate/show_logical_replication_jobs.go +++ b/pkg/sql/delegate/show_logical_replication_jobs.go @@ -53,7 +53,7 @@ WHERE job_type = 'LOGICAL REPLICATION' payload)->'logicalReplicationDetails'->'defaultConflictResolution'->>'conflictResolutionType', 'LWW') AS conflict_resolution_type, crdb_internal.pb_to_json( 'cockroach.sql.jobs.jobspb.Payload', - payload)->>'description' AS description` + payload)->'logicalReplicationDetails'->>'command' AS command` ) func (d *delegator) delegateShowLogicalReplicationJobs( From 7d9b214cda7b0fda89df39522bc1750a7ab96caf Mon Sep 17 00:00:00 2001 From: Aerin Freilich Date: Thu, 19 Dec 2024 10:43:35 -0500 Subject: [PATCH 092/126] ccl/changeedccl: add changefeed options into nemesis tests This work makes sure our nemesis tests for changefeeds randomize over the options we use upon changefeed creation. This randomly adds the key_in_value option (see below) and full_table_name option half of the time and checks that the changefeed messages respect them in the beforeAfter validator. Note the following limitations: the full_table_name option, when on, asserts that the topic in the output will be d.public.{table_name} instead of checking for the actual name of the database/schema. This change also does not add the key_in_value option when for the webhook and cloudstorage sinks. Even before this change, since key_in_value is on by default for those sinks, we remove the key from the value in those testfeed messages for ease of testing. Unfortunately, this makes these cases hard to test, so we leave them out for now. See also: #134119 Epic: CRDB-42866 Release note: None --- pkg/ccl/changefeedccl/cdctest/nemeses.go | 72 ++++++-- pkg/ccl/changefeedccl/cdctest/validator.go | 65 +++++-- .../changefeedccl/cdctest/validator_test.go | 170 ++++++++++-------- pkg/ccl/changefeedccl/nemeses_test.go | 7 +- pkg/ccl/changefeedccl/validations_test.go | 2 +- pkg/cmd/roachtest/tests/cdc.go | 12 +- pkg/cmd/roachtest/tests/mixed_version_cdc.go | 2 +- .../replication_random_client_test.go | 2 +- 8 files changed, 228 insertions(+), 104 deletions(-) diff --git a/pkg/ccl/changefeedccl/cdctest/nemeses.go b/pkg/ccl/changefeedccl/cdctest/nemeses.go index 2d278f4c96bf..ba5f566c3a76 100644 --- a/pkg/ccl/changefeedccl/cdctest/nemeses.go +++ b/pkg/ccl/changefeedccl/cdctest/nemeses.go @@ -19,6 +19,53 @@ import ( "github.com/cockroachdb/errors" ) +type ChangefeedOption struct { + FullTableName bool + Format string + KeyInValue bool +} + +func newChangefeedOption(testName string) ChangefeedOption { + isCloudstorage := strings.Contains(testName, "cloudstorage") + isWebhook := strings.Contains(testName, "webhook") + cfo := ChangefeedOption{ + FullTableName: rand.Intn(2) < 1, + + // Because key_in_value is on by default for cloudstorage and webhook sinks, + // the key in the value is extracted and removed from the test feed + // messages (see extractKeyFromJSONValue function). + // TODO: (#138749) enable testing key_in_value for cloudstorage + // and webhook sinks + KeyInValue: !isCloudstorage && !isWebhook && rand.Intn(2) < 1, + Format: "json", + } + + if isCloudstorage && rand.Intn(2) < 1 { + cfo.Format = "parquet" + } + + return cfo +} + +func (co ChangefeedOption) String() string { + return fmt.Sprintf("full_table_name=%t,key_in_value=%t,format=%s", + co.FullTableName, co.KeyInValue, co.Format) +} + +func (cfo ChangefeedOption) OptionString() string { + options := "" + if cfo.Format == "parquet" { + options = ", format=parquet" + } + if cfo.FullTableName { + options = options + ", full_table_name" + } + if cfo.KeyInValue { + options = options + ", key_in_value" + } + return options +} + type NemesesOption struct { EnableFpValidator bool EnableSQLSmith bool @@ -36,7 +83,8 @@ var NemesesOptions = []NemesesOption{ } func (no NemesesOption) String() string { - return fmt.Sprintf("fp_validator=%t,sql_smith=%t", no.EnableFpValidator, no.EnableSQLSmith) + return fmt.Sprintf("fp_validator=%t,sql_smith=%t", + no.EnableFpValidator, no.EnableSQLSmith) } // RunNemesis runs a jepsen-style validation of whether a changefeed meets our @@ -50,8 +98,7 @@ func (no NemesesOption) String() string { func RunNemesis( f TestFeedFactory, db *gosql.DB, - isSinkless bool, - isCloudstorage bool, + testName string, withLegacySchemaChanger bool, rng *rand.Rand, nOp NemesesOption, @@ -69,6 +116,8 @@ func RunNemesis( ctx := context.Background() eventPauseCount := 10 + + isSinkless := strings.Contains(testName, "sinkless") if isSinkless { // Disable eventPause for sinkless changefeeds because we currently do not // have "correct" pause and unpause mechanisms for changefeeds that aren't @@ -199,11 +248,13 @@ func RunNemesis( } } - withFormatParquet := "" - if isCloudstorage && rand.Intn(2) < 1 { - withFormatParquet = ", format=parquet" - } - foo, err := f.Feed(fmt.Sprintf(`CREATE CHANGEFEED FOR foo WITH updated, resolved, diff %s`, withFormatParquet)) + cfo := newChangefeedOption(testName) + changefeedStatement := fmt.Sprintf( + `CREATE CHANGEFEED FOR foo WITH updated, resolved, diff%s`, + cfo.OptionString(), + ) + log.Infof(ctx, "Using changefeed options: %s", changefeedStatement) + foo, err := f.Feed(changefeedStatement) if err != nil { return nil, err } @@ -218,7 +269,8 @@ func RunNemesis( if _, err := db.Exec(createFprintStmtBuf.String()); err != nil { return nil, err } - baV, err := NewBeforeAfterValidator(db, `foo`) + + baV, err := NewBeforeAfterValidator(db, `foo`, cfo) if err != nil { return nil, err } @@ -817,7 +869,7 @@ func noteFeedMessage(a fsm.Args) error { } ns.availableRows-- log.Infof(a.Ctx, "%s->%s", m.Key, m.Value) - return ns.v.NoteRow(m.Partition, string(m.Key), string(m.Value), ts) + return ns.v.NoteRow(m.Partition, string(m.Key), string(m.Value), ts, m.Topic) } } } diff --git a/pkg/ccl/changefeedccl/cdctest/validator.go b/pkg/ccl/changefeedccl/cdctest/validator.go index 1fe47f0afb5f..f7a2322a62c0 100644 --- a/pkg/ccl/changefeedccl/cdctest/validator.go +++ b/pkg/ccl/changefeedccl/cdctest/validator.go @@ -23,7 +23,7 @@ import ( // guarantees in a single table. type Validator interface { // NoteRow accepts a changed row entry. - NoteRow(partition string, key, value string, updated hlc.Timestamp) error + NoteRow(partition, key, value string, updated hlc.Timestamp, topic string) error // NoteResolved accepts a resolved timestamp entry. NoteResolved(partition string, resolved hlc.Timestamp) error // Failures returns any violations seen so far. @@ -64,7 +64,7 @@ var _ StreamValidator = &orderValidator{} type noOpValidator struct{} // NoteRow accepts a changed row entry. -func (v *noOpValidator) NoteRow(string, string, string, hlc.Timestamp) error { return nil } +func (v *noOpValidator) NoteRow(string, string, string, hlc.Timestamp, string) error { return nil } // NoteResolved accepts a resolved timestamp entry. func (v *noOpValidator) NoteResolved(string, hlc.Timestamp) error { return nil } @@ -125,7 +125,9 @@ func (v *orderValidator) GetValuesForKeyBelowTimestamp( } // NoteRow implements the Validator interface. -func (v *orderValidator) NoteRow(partition string, key, value string, updated hlc.Timestamp) error { +func (v *orderValidator) NoteRow( + partition, key, value string, updated hlc.Timestamp, topic string, +) error { if prev, ok := v.partitionForKey[key]; ok && prev != partition { v.failures = append(v.failures, fmt.Sprintf( `key [%s] received on two partitions: %s and %s`, key, prev, partition, @@ -189,6 +191,8 @@ type beforeAfterValidator struct { table string primaryKeyCols []string resolved map[string]hlc.Timestamp + fullTableName bool + keyInValue bool failures []string } @@ -196,7 +200,9 @@ type beforeAfterValidator struct { // NewBeforeAfterValidator returns a Validator verifies that the "before" and // "after" fields in each row agree with the source table when performing AS OF // SYSTEM TIME lookups before and at the row's timestamp. -func NewBeforeAfterValidator(sqlDB *gosql.DB, table string) (Validator, error) { +func NewBeforeAfterValidator( + sqlDB *gosql.DB, table string, option ChangefeedOption, +) (Validator, error) { primaryKeyCols, err := fetchPrimaryKeyCols(sqlDB, table) if err != nil { return nil, errors.Wrap(err, "fetchPrimaryKeyCols failed") @@ -205,6 +211,8 @@ func NewBeforeAfterValidator(sqlDB *gosql.DB, table string) (Validator, error) { return &beforeAfterValidator{ sqlDB: sqlDB, table: table, + fullTableName: option.FullTableName, + keyInValue: option.KeyInValue, primaryKeyCols: primaryKeyCols, resolved: make(map[string]hlc.Timestamp), }, nil @@ -212,8 +220,21 @@ func NewBeforeAfterValidator(sqlDB *gosql.DB, table string) (Validator, error) { // NoteRow implements the Validator interface. func (v *beforeAfterValidator) NoteRow( - partition string, key, value string, updated hlc.Timestamp, + partition, key, value string, updated hlc.Timestamp, topic string, ) error { + if v.fullTableName { + if topic != fmt.Sprintf(`d.public.%s`, v.table) { + v.failures = append(v.failures, fmt.Sprintf( + "topic %s does not match expected table d.public.%s", topic, v.table, + )) + } + } else { + if topic != v.table { + v.failures = append(v.failures, fmt.Sprintf( + "topic %s does not match expected table %s", topic, v.table, + )) + } + } keyJSON, err := json.ParseJSON(key) if err != nil { return err @@ -230,6 +251,26 @@ func (v *beforeAfterValidator) NoteRow( return err } + if v.keyInValue { + keyString := keyJSON.String() + keyInValueJSON, err := valueJSON.FetchValKey("key") + if err != nil { + return err + } + + if keyInValueJSON == nil { + v.failures = append(v.failures, fmt.Sprintf( + "no key in value, expected key value %s", keyString)) + } else { + keyInValueString := keyInValueJSON.String() + if keyInValueString != keyString { + v.failures = append(v.failures, fmt.Sprintf( + "key in value %s does not match expected key value %s", + keyInValueString, keyString)) + } + } + } + afterJSON, err := valueJSON.FetchValKey("after") if err != nil { return err @@ -451,7 +492,7 @@ func (v *FingerprintValidator) DBFunc( // NoteRow implements the Validator interface. func (v *FingerprintValidator) NoteRow( - ignoredPartition string, key, value string, updated hlc.Timestamp, + partition, key, value string, updated hlc.Timestamp, topic string, ) error { if v.firstRowTimestamp.IsEmpty() || updated.Less(v.firstRowTimestamp) { v.firstRowTimestamp = updated @@ -663,9 +704,11 @@ func (v *FingerprintValidator) Failures() []string { type Validators []Validator // NoteRow implements the Validator interface. -func (vs Validators) NoteRow(partition string, key, value string, updated hlc.Timestamp) error { +func (vs Validators) NoteRow( + partition, key, value string, updated hlc.Timestamp, topic string, +) error { for _, v := range vs { - if err := v.NoteRow(partition, key, value, updated); err != nil { + if err := v.NoteRow(partition, key, value, updated, topic); err != nil { return err } } @@ -707,10 +750,12 @@ func NewCountValidator(v Validator) *CountValidator { } // NoteRow implements the Validator interface. -func (v *CountValidator) NoteRow(partition string, key, value string, updated hlc.Timestamp) error { +func (v *CountValidator) NoteRow( + partition, key, value string, updated hlc.Timestamp, topic string, +) error { v.NumRows++ v.rowsSinceResolved++ - return v.v.NoteRow(partition, key, value, updated) + return v.v.NoteRow(partition, key, value, updated, topic) } // NoteResolved implements the Validator interface. diff --git a/pkg/ccl/changefeedccl/cdctest/validator_test.go b/pkg/ccl/changefeedccl/cdctest/validator_test.go index 922a50be3f86..ae8fa8ba85ad 100644 --- a/pkg/ccl/changefeedccl/cdctest/validator_test.go +++ b/pkg/ccl/changefeedccl/cdctest/validator_test.go @@ -24,9 +24,13 @@ func ts(i int64) hlc.Timestamp { return hlc.Timestamp{WallTime: i} } -func noteRow(t *testing.T, v Validator, partition, key, value string, updated hlc.Timestamp) { +func noteRow( + t *testing.T, v Validator, partition, key, value string, updated hlc.Timestamp, topic string, +) { t.Helper() - if err := v.NoteRow(partition, key, value, updated); err != nil { + // None of the validators in this file include assertions about the topic + // name, so it's ok to pass in an empty string for topic. + if err := v.NoteRow(partition, key, value, updated, topic); err != nil { t.Fatal(err) } } @@ -57,23 +61,23 @@ func TestOrderValidator(t *testing.T) { }) t.Run(`dupe okay`, func(t *testing.T) { v := NewOrderValidator(`t1`) - noteRow(t, v, `p1`, `k1`, ignored, ts(1)) - noteRow(t, v, `p1`, `k1`, ignored, ts(2)) - noteRow(t, v, `p1`, `k1`, ignored, ts(1)) + noteRow(t, v, `p1`, `k1`, ignored, ts(1), `foo`) + noteRow(t, v, `p1`, `k1`, ignored, ts(2), `foo`) + noteRow(t, v, `p1`, `k1`, ignored, ts(1), `foo`) assertValidatorFailures(t, v) }) t.Run(`key on two partitions`, func(t *testing.T) { v := NewOrderValidator(`t1`) - noteRow(t, v, `p1`, `k1`, ignored, ts(2)) - noteRow(t, v, `p2`, `k1`, ignored, ts(1)) + noteRow(t, v, `p1`, `k1`, ignored, ts(2), `foo`) + noteRow(t, v, `p2`, `k1`, ignored, ts(1), `foo`) assertValidatorFailures(t, v, `key [k1] received on two partitions: p1 and p2`, ) }) t.Run(`new key with lower timestamp`, func(t *testing.T) { v := NewOrderValidator(`t1`) - noteRow(t, v, `p1`, `k1`, ignored, ts(2)) - noteRow(t, v, `p1`, `k1`, ignored, ts(1)) + noteRow(t, v, `p1`, `k1`, ignored, ts(2), `foo`) + noteRow(t, v, `p1`, `k1`, ignored, ts(1), `foo`) assertValidatorFailures(t, v, `topic t1 partition p1: saw new row timestamp 1.0000000000 after 2.0000000000 was seen`, ) @@ -82,12 +86,12 @@ func TestOrderValidator(t *testing.T) { v := NewOrderValidator(`t1`) noteResolved(t, v, `p2`, ts(3)) // Okay because p2 saw the resolved timestamp but p1 didn't. - noteRow(t, v, `p1`, `k1`, ignored, ts(1)) + noteRow(t, v, `p1`, `k1`, ignored, ts(1), `foo`) noteResolved(t, v, `p1`, ts(3)) // This one is not okay. - noteRow(t, v, `p1`, `k1`, ignored, ts(2)) + noteRow(t, v, `p1`, `k1`, ignored, ts(2), `foo`) // Still okay because we've seen it before. - noteRow(t, v, `p1`, `k1`, ignored, ts(1)) + noteRow(t, v, `p1`, `k1`, ignored, ts(1), `foo`) assertValidatorFailures(t, v, `topic t1 partition p1`+ `: saw new row timestamp 2.0000000000 after 3.0000000000 was resolved`, @@ -95,6 +99,12 @@ func TestOrderValidator(t *testing.T) { }) } +var standardChangefeedOptions = ChangefeedOption{ + FullTableName: false, + KeyInValue: false, + Format: "json", +} + func TestBeforeAfterValidator(t *testing.T) { defer leaktest.AfterTest(t)() @@ -130,97 +140,115 @@ func TestBeforeAfterValidator(t *testing.T) { } t.Run(`empty`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) + require.NoError(t, err) + assertValidatorFailures(t, v) + }) + t.Run(`fullTableName`, func(t *testing.T) { + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, ChangefeedOption{ + FullTableName: true, + KeyInValue: false, + Format: "json", + }) + require.NoError(t, err) + assertValidatorFailures(t, v) + }) + t.Run(`key_in_value`, func(t *testing.T) { + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, ChangefeedOption{ + FullTableName: false, + KeyInValue: true, + Format: "json", + }) require.NoError(t, err) assertValidatorFailures(t, v) }) t.Run(`during initial`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) // "before" is ignored if missing. - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1]) - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1], `foo`) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2], `foo`) // However, if provided, it is validated. - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":2}, "before": {"k":1,"v":1}}`, ts[2]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":2}, "before": {"k":1,"v":1}}`, ts[2], `foo`) assertValidatorFailures(t, v) - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":3}, "before": {"k":1,"v":3}}`, ts[3]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":3}, "before": {"k":1,"v":3}}`, ts[3], `foo`) assertValidatorFailures(t, v, `"before" field did not agree with row at `+ts[3].Prev().AsOfSystemTime()+ `: SELECT count(*) = 1 FROM foo AS OF SYSTEM TIME '`+ts[3].Prev().AsOfSystemTime()+ `' WHERE to_json(k)::TEXT = $1 AND to_json(v)::TEXT = $2 [1 3]`) }) t.Run(`missing before`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) // "before" should have been provided. - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2], `foo`) assertValidatorFailures(t, v, `"before" field did not agree with row at `+ts[2].Prev().AsOfSystemTime()+ `: SELECT count(*) = 0 FROM foo AS OF SYSTEM TIME '`+ts[2].Prev().AsOfSystemTime()+ `' WHERE to_json(k)::TEXT = $1 [1]`) }) t.Run(`incorrect before`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) // "before" provided with wrong value. - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":3}, "before": {"k":5,"v":10}}`, ts[3]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":3}, "before": {"k":5,"v":10}}`, ts[3], `foo`) assertValidatorFailures(t, v, `"before" field did not agree with row at `+ts[3].Prev().AsOfSystemTime()+ `: SELECT count(*) = 1 FROM foo AS OF SYSTEM TIME '`+ts[3].Prev().AsOfSystemTime()+ `' WHERE to_json(k)::TEXT = $1 AND to_json(v)::TEXT = $2 [5 10]`) }) t.Run(`unnecessary before`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) // "before" provided but should not have been. - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":1}, "before": {"k":1,"v":1}}`, ts[1]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":1}, "before": {"k":1,"v":1}}`, ts[1], `foo`) assertValidatorFailures(t, v, `"before" field did not agree with row at `+ts[1].Prev().AsOfSystemTime()+ `: SELECT count(*) = 1 FROM foo AS OF SYSTEM TIME '`+ts[1].Prev().AsOfSystemTime()+ `' WHERE to_json(k)::TEXT = $1 AND to_json(v)::TEXT = $2 [1 1]`) }) t.Run(`missing after`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) // "after" should have been provided. - noteRow(t, v, `p`, `[1]`, `{"before": {"k":1,"v":1}}`, ts[2]) + noteRow(t, v, `p`, `[1]`, `{"before": {"k":1,"v":1}}`, ts[2], `foo`) assertValidatorFailures(t, v, `"after" field did not agree with row at `+ts[2].AsOfSystemTime()+ `: SELECT count(*) = 0 FROM foo AS OF SYSTEM TIME '`+ts[2].AsOfSystemTime()+ `' WHERE to_json(k)::TEXT = $1 [1]`) }) t.Run(`incorrect after`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) // "after" provided with wrong value. - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":5}, "before": {"k":1,"v":2}}`, ts[3]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":5}, "before": {"k":1,"v":2}}`, ts[3], `foo`) assertValidatorFailures(t, v, `"after" field did not agree with row at `+ts[3].AsOfSystemTime()+ `: SELECT count(*) = 1 FROM foo AS OF SYSTEM TIME '`+ts[3].AsOfSystemTime()+ `' WHERE to_json(k)::TEXT = $1 AND to_json(v)::TEXT = $2 [1 5]`) }) t.Run(`unnecessary after`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) // "after" provided but should not have been. - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":3}, "before": {"k":1,"v":3}}`, ts[4]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":3}, "before": {"k":1,"v":3}}`, ts[4], `foo`) assertValidatorFailures(t, v, `"after" field did not agree with row at `+ts[4].AsOfSystemTime()+ `: SELECT count(*) = 1 FROM foo AS OF SYSTEM TIME '`+ts[4].AsOfSystemTime()+ `' WHERE to_json(k)::TEXT = $1 AND to_json(v)::TEXT = $2 [1 3]`) }) t.Run(`incorrect before and after`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) // "before" and "after" both provided with wrong value. - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":5}, "before": {"k":1,"v":4}}`, ts[3]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":5}, "before": {"k":1,"v":4}}`, ts[3], `foo`) assertValidatorFailures(t, v, `"after" field did not agree with row at `+ts[3].AsOfSystemTime()+ `: SELECT count(*) = 1 FROM foo AS OF SYSTEM TIME '`+ts[3].AsOfSystemTime()+ @@ -230,19 +258,19 @@ func TestBeforeAfterValidator(t *testing.T) { `' WHERE to_json(k)::TEXT = $1 AND to_json(v)::TEXT = $2 [1 4]`) }) t.Run(`correct`, func(t *testing.T) { - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) - noteRow(t, v, `p`, `[1]`, `{}`, ts[0]) - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1]) - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":1}, "before": null}`, ts[1]) - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":2}, "before": {"k":1,"v":1}}`, ts[2]) - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":3}, "before": {"k":1,"v":2}}`, ts[3]) - noteRow(t, v, `p`, `[1]`, `{ "before": {"k":1,"v":3}}`, ts[4]) - noteRow(t, v, `p`, `[1]`, `{"after": null, "before": {"k":1,"v":3}}`, ts[4]) - noteRow(t, v, `p`, `[2]`, `{}`, ts[1]) - noteRow(t, v, `p`, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2]) - noteRow(t, v, `p`, `[2]`, `{"after": {"k":2,"v":2}, "before": null}`, ts[2]) + noteRow(t, v, `p`, `[1]`, `{}`, ts[0], `foo`) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1], `foo`) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":1}, "before": null}`, ts[1], `foo`) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":2}, "before": {"k":1,"v":1}}`, ts[2], `foo`) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1,"v":3}, "before": {"k":1,"v":2}}`, ts[3], `foo`) + noteRow(t, v, `p`, `[1]`, `{ "before": {"k":1,"v":3}}`, ts[4], `foo`) + noteRow(t, v, `p`, `[1]`, `{"after": null, "before": {"k":1,"v":3}}`, ts[4], `foo`) + noteRow(t, v, `p`, `[2]`, `{}`, ts[1], `foo`) + noteRow(t, v, `p`, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2], `foo`) + noteRow(t, v, `p`, `[2]`, `{"after": {"k":2,"v":2}, "before": null}`, ts[2], `foo`) assertValidatorFailures(t, v) }) } @@ -269,10 +297,10 @@ func TestBeforeAfterValidatorForGeometry(t *testing.T) { t.Fatal(err) } } - v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`) + v, err := NewBeforeAfterValidator(sqlDBRaw, `foo`, standardChangefeedOptions) require.NoError(t, err) assertValidatorFailures(t, v) - noteRow(t, v, `p`, `[1]`, `{"after": {"k":1, "geom":{"coordinates": [1,2], "type": "Point"}}}`, ts[0]) + noteRow(t, v, `p`, `[1]`, `{"after": {"k":1, "geom":{"coordinates": [1,2], "type": "Point"}}}`, ts[0], `foo`) } func TestFingerprintValidator(t *testing.T) { @@ -326,7 +354,7 @@ func TestFingerprintValidator(t *testing.T) { sqlDB.Exec(t, createTableStmt(`wrong_data`)) v, err := NewFingerprintValidator(sqlDBRaw, `foo`, `wrong_data`, []string{`p`}, testColumns) require.NoError(t, err) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":10}}`, ts[1]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":10}}`, ts[1], `foo`) noteResolved(t, v, `p`, ts[1]) assertValidatorFailures(t, v, `fingerprints did not match at `+ts[1].AsOfSystemTime()+ @@ -340,14 +368,14 @@ func TestFingerprintValidator(t *testing.T) { if err := v.NoteResolved(`p`, ts[0]); err != nil { t.Fatal(err) } - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1], `foo`) noteResolved(t, v, `p`, ts[1]) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2]) - noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2], `foo`) + noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2], `foo`) noteResolved(t, v, `p`, ts[2]) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":3}}`, ts[3]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":3}}`, ts[3], `foo`) noteResolved(t, v, `p`, ts[3]) - noteRow(t, v, ignored, `[1]`, `{"after": null}`, ts[4]) + noteRow(t, v, ignored, `[1]`, `{"after": null}`, ts[4], `foo`) noteResolved(t, v, `p`, ts[4]) noteResolved(t, v, `p`, ts[5]) assertValidatorFailures(t, v) @@ -356,11 +384,11 @@ func TestFingerprintValidator(t *testing.T) { sqlDB.Exec(t, createTableStmt(`rows_unsorted`)) v, err := NewFingerprintValidator(sqlDBRaw, `foo`, `rows_unsorted`, []string{`p`}, testColumns) require.NoError(t, err) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":3}}`, ts[3]) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2]) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1]) - noteRow(t, v, ignored, `[1]`, `{"after": null}`, ts[4]) - noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":3}}`, ts[3], `foo`) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2], `foo`) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1], `foo`) + noteRow(t, v, ignored, `[1]`, `{"after": null}`, ts[4], `foo`) + noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2], `foo`) noteResolved(t, v, `p`, ts[5]) assertValidatorFailures(t, v) }) @@ -371,9 +399,9 @@ func TestFingerprintValidator(t *testing.T) { noteResolved(t, v, `p`, ts[0]) // Intentionally missing {"k":1,"v":1} at ts[1]. // Insert a fake row since we don't fingerprint earlier than the first seen row. - noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2].Prev()) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2]) - noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2]) + noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2].Prev(), `foo`) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2], `foo`) + noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2], `foo`) noteResolved(t, v, `p`, ts[2].Prev()) assertValidatorFailures(t, v, `fingerprints did not match at `+ts[2].Prev().AsOfSystemTime()+ @@ -385,11 +413,11 @@ func TestFingerprintValidator(t *testing.T) { v, err := NewFingerprintValidator(sqlDBRaw, `foo`, `missed_middle`, []string{`p`}, testColumns) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1], `foo`) // Intentionally missing {"k":1,"v":2} at ts[2]. - noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2]) + noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2], `foo`) noteResolved(t, v, `p`, ts[2]) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":3}}`, ts[3]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":3}}`, ts[3], `foo`) noteResolved(t, v, `p`, ts[3]) assertValidatorFailures(t, v, `fingerprints did not match at `+ts[2].AsOfSystemTime()+ @@ -403,9 +431,9 @@ func TestFingerprintValidator(t *testing.T) { v, err := NewFingerprintValidator(sqlDBRaw, `foo`, `missed_end`, []string{`p`}, testColumns) require.NoError(t, err) noteResolved(t, v, `p`, ts[0]) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1]) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2]) - noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1], `foo`) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2], `foo`) + noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[2], `foo`) // Intentionally missing {"k":1,"v":3} at ts[3]. noteResolved(t, v, `p`, ts[3]) assertValidatorFailures(t, v, @@ -417,8 +445,8 @@ func TestFingerprintValidator(t *testing.T) { sqlDB.Exec(t, createTableStmt(`initial_scan`)) v, err := NewFingerprintValidator(sqlDBRaw, `foo`, `initial_scan`, []string{`p`}, testColumns) require.NoError(t, err) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":3}}`, ts[3]) - noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[3]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":3}}`, ts[3], `foo`) + noteRow(t, v, ignored, `[2]`, `{"after": {"k":2,"v":2}}`, ts[3], `foo`) noteResolved(t, v, `p`, ts[3]) assertValidatorFailures(t, v) }) @@ -434,7 +462,7 @@ func TestFingerprintValidator(t *testing.T) { sqlDB.Exec(t, createTableStmt(`resolved_unsorted`)) v, err := NewFingerprintValidator(sqlDBRaw, `foo`, `resolved_unsorted`, []string{`p`}, testColumns) require.NoError(t, err) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1], `foo`) noteResolved(t, v, `p`, ts[1]) noteResolved(t, v, `p`, ts[1]) noteResolved(t, v, `p`, ts[0]) @@ -444,8 +472,8 @@ func TestFingerprintValidator(t *testing.T) { sqlDB.Exec(t, createTableStmt(`two_partitions`)) v, err := NewFingerprintValidator(sqlDBRaw, `foo`, `two_partitions`, []string{`p0`, `p1`}, testColumns) require.NoError(t, err) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1]) - noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2]) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":1}}`, ts[1], `foo`) + noteRow(t, v, ignored, `[1]`, `{"after": {"k":1,"v":2}}`, ts[2], `foo`) // Intentionally missing {"k":2,"v":2}. noteResolved(t, v, `p0`, ts[2]) noteResolved(t, v, `p0`, ts[4]) @@ -478,7 +506,7 @@ func TestValidators(t *testing.T) { NewOrderValidator(`t2`), } noteResolved(t, v, `p1`, ts(2)) - noteRow(t, v, `p1`, `k1`, ignored, ts(1)) + noteRow(t, v, `p1`, `k1`, ignored, ts(1), `foo`) assertValidatorFailures(t, v, `topic t1 partition p1`+ `: saw new row timestamp 1.0000000000 after 2.0000000000 was resolved`, diff --git a/pkg/ccl/changefeedccl/nemeses_test.go b/pkg/ccl/changefeedccl/nemeses_test.go index 4c2bd7975b6b..3386784cf7c7 100644 --- a/pkg/ccl/changefeedccl/nemeses_test.go +++ b/pkg/ccl/changefeedccl/nemeses_test.go @@ -8,7 +8,6 @@ package changefeedccl import ( "math" "regexp" - "strings" "testing" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest" @@ -35,12 +34,8 @@ func TestChangefeedNemeses(t *testing.T) { sqlDB := sqlutils.MakeSQLRunner(s.DB) withLegacySchemaChanger := maybeDisableDeclarativeSchemaChangesForTest(t, sqlDB) - // TODO(dan): Ugly hack to disable `eventPause` in sinkless feeds. See comment in - // `RunNemesis` for details. - isSinkless := strings.Contains(t.Name(), "sinkless") - isCloudstorage := strings.Contains(t.Name(), "cloudstorage") - v, err := cdctest.RunNemesis(f, s.DB, isSinkless, isCloudstorage, withLegacySchemaChanger, rng, nop) + v, err := cdctest.RunNemesis(f, s.DB, t.Name(), withLegacySchemaChanger, rng, nop) if err != nil { t.Fatalf("%+v", err) } diff --git a/pkg/ccl/changefeedccl/validations_test.go b/pkg/ccl/changefeedccl/validations_test.go index 4a3ca153a022..32f28fec337a 100644 --- a/pkg/ccl/changefeedccl/validations_test.go +++ b/pkg/ccl/changefeedccl/validations_test.go @@ -75,7 +75,7 @@ func TestCatchupScanOrdering(t *testing.T) { if err != nil { t.Fatal(err) } - err = v.NoteRow(m.Partition, string(m.Key), string(m.Value), updated) + err = v.NoteRow(m.Partition, string(m.Key), string(m.Value), updated, m.Topic) if err != nil { t.Fatal(err) } diff --git a/pkg/cmd/roachtest/tests/cdc.go b/pkg/cmd/roachtest/tests/cdc.go index 3cdde1b96141..7359b210bc87 100644 --- a/pkg/cmd/roachtest/tests/cdc.go +++ b/pkg/cmd/roachtest/tests/cdc.go @@ -898,7 +898,7 @@ func runCDCBank(ctx context.Context, t test.Test, c cluster.Cluster) { partitionStr := strconv.Itoa(int(m.Partition)) if len(m.Key) > 0 { - if err := v.NoteRow(partitionStr, string(m.Key), string(m.Value), updated); err != nil { + if err := v.NoteRow(partitionStr, string(m.Key), string(m.Value), updated, m.Topic); err != nil { return err } } else { @@ -926,7 +926,11 @@ func runCDCBank(ctx context.Context, t test.Test, c cluster.Cluster) { if err != nil { return errors.Wrap(err, "error creating validator") } - baV, err := cdctest.NewBeforeAfterValidator(db, `bank.bank`) + baV, err := cdctest.NewBeforeAfterValidator(db, `bank.bank`, cdctest.ChangefeedOption{ + FullTableName: false, + KeyInValue: false, + Format: "json", + }) if err != nil { return err } @@ -953,7 +957,7 @@ func runCDCBank(ctx context.Context, t test.Test, c cluster.Cluster) { partitionStr := strconv.Itoa(int(m.Partition)) if len(m.Key) > 0 { startTime := timeutil.Now() - if err := v.NoteRow(partitionStr, string(m.Key), string(m.Value), updated); err != nil { + if err := v.NoteRow(partitionStr, string(m.Key), string(m.Value), updated, m.Topic); err != nil { return err } timeSpentValidatingRows += timeutil.Since(startTime) @@ -3890,7 +3894,7 @@ func (c *topicConsumer) validateMessage(partition int32, m *sarama.ConsumerMessa return err } default: - err := c.validator.NoteRow(partitionStr, string(m.Key), string(m.Value), updated) + err := c.validator.NoteRow(partitionStr, string(m.Key), string(m.Value), updated, m.Topic) if err != nil { return err } diff --git a/pkg/cmd/roachtest/tests/mixed_version_cdc.go b/pkg/cmd/roachtest/tests/mixed_version_cdc.go index c62b7d9d34c1..230ecbd83aa3 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_cdc.go +++ b/pkg/cmd/roachtest/tests/mixed_version_cdc.go @@ -301,7 +301,7 @@ func (cmvt *cdcMixedVersionTester) validate( partitionStr := strconv.Itoa(int(m.Partition)) if len(m.Key) > 0 { - if err := cmvt.validator.NoteRow(partitionStr, string(m.Key), string(m.Value), updated); err != nil { + if err := cmvt.validator.NoteRow(partitionStr, string(m.Key), string(m.Value), updated, m.Topic); err != nil { return err } } else { diff --git a/pkg/crosscluster/physical/replication_random_client_test.go b/pkg/crosscluster/physical/replication_random_client_test.go index 21d908d69f45..f9740a73f4da 100644 --- a/pkg/crosscluster/physical/replication_random_client_test.go +++ b/pkg/crosscluster/physical/replication_random_client_test.go @@ -95,7 +95,7 @@ func (sv *streamClientValidator) noteRow( ) error { sv.mu.Lock() defer sv.mu.Unlock() - return sv.NoteRow(partition, key, value, updated) + return sv.NoteRow(partition, key, value, updated, "" /* topic */) } func (sv *streamClientValidator) noteResolved(partition string, resolved hlc.Timestamp) error { From c8d3af79b2924c5027a99e0d74ab53aa688c583e Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 14 Jan 2025 16:53:33 +0100 Subject: [PATCH 093/126] testutils,kvserver: add StartExecTrace and adopt in TestPromoteNonVoterInAddVoter Touches https://github.com/cockroachdb/cockroach/issues/134383. Epic: None Release note: None --- pkg/kv/kvserver/replicate_queue_test.go | 5 +- pkg/testutils/BUILD.bazel | 2 + pkg/testutils/exectrace.go | 71 +++++++++++++++++++++++++ 3 files changed, 77 insertions(+), 1 deletion(-) create mode 100644 pkg/testutils/exectrace.go diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index 5b221d7cd463..893466b802ee 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -2140,7 +2140,8 @@ func iterateOverAllStores( // the range log where the added replica type is a LEARNER. func TestPromoteNonVoterInAddVoter(t *testing.T) { defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) + scope := log.Scope(t) + defer scope.Close(t) // This test is slow under stress/race and can time out when upreplicating / // rebalancing to ensure all stores have the same range count initially, due @@ -2149,6 +2150,8 @@ func TestPromoteNonVoterInAddVoter(t *testing.T) { skip.UnderDeadlock(t) skip.UnderRace(t) + defer testutils.StartExecTrace(t, scope.GetDirectory()).Finish(t) + ctx := context.Background() // Create 7 stores: 3 in Region 1, 2 in Region 2, and 2 in Region 3. diff --git a/pkg/testutils/BUILD.bazel b/pkg/testutils/BUILD.bazel index 5fb1beba8f2e..f0fcdb14104c 100644 --- a/pkg/testutils/BUILD.bazel +++ b/pkg/testutils/BUILD.bazel @@ -6,6 +6,7 @@ go_library( "backup.go", "dir.go", "error.go", + "exectrace.go", "files.go", "hook.go", "keys.go", @@ -33,6 +34,7 @@ go_library( "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", "@com_github_dataexmachina_dev_side_eye_go//sideeye", + "@com_github_petermattis_goid//:goid", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/testutils/exectrace.go b/pkg/testutils/exectrace.go new file mode 100644 index 000000000000..c9c4e60dfad5 --- /dev/null +++ b/pkg/testutils/exectrace.go @@ -0,0 +1,71 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package testutils + +import ( + "context" + "fmt" + "os" + "path/filepath" + "runtime/trace" + + "github.com/petermattis/goid" +) + +type ActiveExecTrace struct { + name string + file *os.File + reg *trace.Region +} + +// Finish stops the ongoing execution trace, if there is one, and closes the +// file. It must be called only once. +func (a *ActiveExecTrace) Finish(t interface { + Failed() bool + Logf(string, ...interface{}) +}) { + if a == nil { + return + } + a.reg.End() + trace.Stop() + _ = a.file.Close() + if !t.Failed() { + _ = os.Remove(a.file.Name()) + } else { + t.Logf("execution trace written to %s", a.file.Name()) + } +} + +// StartExecTrace starts a Go execution trace and returns a handle that allows +// stopping it. If a trace cannot be started, this is logged and nil is returned. +// It is valid to stop a nil ActiveExecTrace. +// +// This helper is intended to instrument tests for which an execution trace is +// desired on the next failure. +func StartExecTrace( + t interface { + Name() string + Logf(string, ...interface{}) + }, dir string, +) *ActiveExecTrace { + path := filepath.Join(dir, fmt.Sprintf("exectrace_goid_%d.bin", goid.Get())) + f, err := os.Create(path) + if err != nil { + t.Logf("could not create file for execution trace: %s", err) + return nil + } + if err := trace.Start(f); err != nil { + _ = f.Close() + t.Logf("could not start execution trace: %s", err) + return nil + } + return &ActiveExecTrace{ + name: t.Name(), + file: f, + reg: trace.StartRegion(context.Background(), t.Name()), + } +} From 4414a36dab51b4988d7e3ea7baee7c20ae9fd834 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Tue, 14 Jan 2025 11:19:32 -0500 Subject: [PATCH 094/126] ccl/serverccl: revise `TestTenantVars` cpu time checks Previously, this test verified that a portion of the test's user cpu time would be less than or equal to the entire tenant user cpu time up to that point. This check is flaky because there's no guarantee that the inactive tenant's cpu time will surpass the test cpu time. We now simply verify that the test cpu times are greater than or equal to the tenant metrics. The test was likely passing before 331596c because the reported tenant cpu time was accounting for the sql server prestart. A tenant's user cpu metrics are tracked from the time the `_status/load` endpoint is registered, and the commit above moved the router setup to occur just after the prestart. Epic: none Fixes: #119329 Release note: None --- pkg/ccl/serverccl/tenant_vars_test.go | 66 +++++++++++++-------------- 1 file changed, 32 insertions(+), 34 deletions(-) diff --git a/pkg/ccl/serverccl/tenant_vars_test.go b/pkg/ccl/serverccl/tenant_vars_test.go index 0e16b68ec1ac..665341dd0ce0 100644 --- a/pkg/ccl/serverccl/tenant_vars_test.go +++ b/pkg/ccl/serverccl/tenant_vars_test.go @@ -69,17 +69,17 @@ func TestTenantVars(t *testing.T) { metrics, err := parser.TextToMetricFamilies(resp.Body) require.NoError(t, err) - userCPU, found := metrics["sys_cpu_user_ns"] + tenantUserCPU, found := metrics["sys_cpu_user_ns"] require.True(t, found) - require.Len(t, userCPU.GetMetric(), 1) - require.Equal(t, io_prometheus_client.MetricType_GAUGE, userCPU.GetType()) - cpuUserNanos := userCPU.Metric[0].GetGauge().GetValue() + require.Len(t, tenantUserCPU.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, tenantUserCPU.GetType()) + tenantCpuUserNanos := tenantUserCPU.Metric[0].GetGauge().GetValue() - sysCPU, found := metrics["sys_cpu_sys_ns"] + tenantSysCPU, found := metrics["sys_cpu_sys_ns"] require.True(t, found) - require.Len(t, sysCPU.GetMetric(), 1) - require.Equal(t, io_prometheus_client.MetricType_GAUGE, sysCPU.GetType()) - cpuSysNanos := sysCPU.Metric[0].GetGauge().GetValue() + require.Len(t, tenantSysCPU.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, tenantSysCPU.GetType()) + tenantCpuSysNanos := tenantSysCPU.Metric[0].GetGauge().GetValue() now, found := metrics["sys_cpu_now_ns"] require.True(t, found) @@ -87,23 +87,22 @@ func TestTenantVars(t *testing.T) { require.Equal(t, io_prometheus_client.MetricType_GAUGE, now.GetType()) nowNanos := now.Metric[0].GetGauge().GetValue() - uptime, found := metrics["sys_uptime"] + tenantUptime, found := metrics["sys_uptime"] require.True(t, found) - require.Len(t, uptime.GetMetric(), 1) - require.Equal(t, io_prometheus_client.MetricType_GAUGE, uptime.GetType()) - uptimeSeconds := uptime.Metric[0].GetGauge().GetValue() + require.Len(t, tenantUptime.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, tenantUptime.GetType()) + uptimeSeconds := tenantUptime.Metric[0].GetGauge().GetValue() // The values are between zero and whatever User/Sys time is observed after the get. require.LessOrEqual(t, float64(startNowNanos), nowNanos) require.LessOrEqual(t, nowNanos, float64(timeutil.Now().UnixNano())) - cpuTime := gosigar.ProcTime{} - require.NoError(t, cpuTime.Get(os.Getpid())) - require.LessOrEqual(t, 0., cpuUserNanos) - require.LessOrEqual(t, cpuUserNanos, float64(cpuTime.User)*1e6) - require.LessOrEqual(t, 0., cpuSysNanos) - require.LessOrEqual(t, cpuSysNanos, float64(cpuTime.Sys)*1e6) - + testCpuTime := gosigar.ProcTime{} + require.NoError(t, testCpuTime.Get(os.Getpid())) + require.LessOrEqual(t, 0., tenantCpuUserNanos) + require.LessOrEqual(t, tenantCpuUserNanos, float64(testCpuTime.User)*1e6) + require.LessOrEqual(t, 0., tenantCpuSysNanos) + require.LessOrEqual(t, tenantCpuSysNanos, float64(testCpuTime.Sys)*1e6) require.LessOrEqual(t, 0., uptimeSeconds) resp, err = client.Get(url) @@ -115,30 +114,29 @@ func TestTenantVars(t *testing.T) { metrics, err = parser.TextToMetricFamilies(resp.Body) require.NoError(t, err) - userCPU, found = metrics["sys_cpu_user_ns"] + tenantUserCPU, found = metrics["sys_cpu_user_ns"] require.True(t, found) - require.Len(t, userCPU.GetMetric(), 1) - require.Equal(t, io_prometheus_client.MetricType_GAUGE, userCPU.GetType()) - cpuUserNanos2 := userCPU.Metric[0].GetGauge().GetValue() + require.Len(t, tenantUserCPU.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, tenantUserCPU.GetType()) + tenantCpuUserNanos2 := tenantUserCPU.Metric[0].GetGauge().GetValue() - sysCPU, found = metrics["sys_cpu_sys_ns"] + tenantSysCPU, found = metrics["sys_cpu_sys_ns"] require.True(t, found) - require.Len(t, sysCPU.GetMetric(), 1) - require.Equal(t, io_prometheus_client.MetricType_GAUGE, sysCPU.GetType()) - // cpuSysNanos2 := sysCPU.Metric[0].GetGauge().GetValue() + require.Len(t, tenantSysCPU.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, tenantSysCPU.GetType()) + tenantCpuSysNanos2 := tenantSysCPU.Metric[0].GetGauge().GetValue() - uptime, found = metrics["sys_uptime"] + tenantUptime, found = metrics["sys_uptime"] require.True(t, found) - require.Len(t, uptime.GetMetric(), 1) - require.Equal(t, io_prometheus_client.MetricType_GAUGE, uptime.GetType()) - uptimeSeconds2 := uptime.Metric[0].GetGauge().GetValue() + require.Len(t, tenantUptime.GetMetric(), 1) + require.Equal(t, io_prometheus_client.MetricType_GAUGE, tenantUptime.GetType()) + uptimeSeconds2 := tenantUptime.Metric[0].GetGauge().GetValue() cpuTime2 := gosigar.ProcTime{} require.NoError(t, cpuTime2.Get(os.Getpid())) - require.LessOrEqual(t, float64(cpuTime2.User-cpuTime.User)*1e6, cpuUserNanos2) - // TODO(#119329): Sometimes our metrics have 0 cpuSysNanos. - // require.LessOrEqual(t, float64(cpuTime2.Sys-cpuTime.Sys)*1e6, cpuSysNanos2) + require.Less(t, tenantCpuUserNanos2, float64(cpuTime2.User)*1e6) + require.LessOrEqual(t, tenantCpuSysNanos2, float64(cpuTime2.Sys)*1e6) require.LessOrEqual(t, uptimeSeconds, uptimeSeconds2) _, found = metrics["jobs_running_non_idle"] From 812ae9898153833c8fe1f93c58c34d0198e3ef30 Mon Sep 17 00:00:00 2001 From: Miles Frankel Date: Mon, 6 Jan 2025 14:24:14 -0500 Subject: [PATCH 095/126] changefeedccl: fix PTS test Fix failing TestPTSRecordProtectsTargetsAndSystemTables test Fixes: #135639 Fixes: #138066 Fixes: #137885 Fixes: #137505 Fixes: #136396 Fixes: #135805 Fixes: #135639 Release note: None --- pkg/ccl/changefeedccl/BUILD.bazel | 1 + pkg/ccl/changefeedccl/changefeed_dist.go | 4 +- .../protected_timestamps_test.go | 70 +++++++++++++++++-- pkg/spanconfig/spanconfigjob/job.go | 12 ++-- .../spanconfigkvaccessor/kvaccessor.go | 2 + .../spanconfigmanager/manager_test.go | 4 +- .../spanconfigreconciler/reconciler.go | 4 ++ pkg/spanconfig/spanconfigstore/store.go | 2 +- pkg/spanconfig/testing_knobs.go | 2 +- 9 files changed, 84 insertions(+), 17 deletions(-) diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index 6e7b4a840b5b..9888128b4b21 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -289,6 +289,7 @@ go_test( "//pkg/server/telemetry", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigjob", "//pkg/spanconfig/spanconfigptsreader", "//pkg/sql", "//pkg/sql/catalog", diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go index d2db50babc54..ee3c2f092483 100644 --- a/pkg/ccl/changefeedccl/changefeed_dist.go +++ b/pkg/ccl/changefeedccl/changefeed_dist.go @@ -149,7 +149,7 @@ func fetchTableDescriptors( ) error { targetDescs = make([]catalog.TableDescriptor, 0, targets.NumUniqueTables()) if err := txn.KV().SetFixedTimestamp(ctx, ts); err != nil { - return err + return errors.Wrapf(err, "setting timestamp for table descriptor fetch") } // Note that all targets are currently guaranteed to have a Table ID // and lie within the primary index span. Deduplication is important @@ -157,7 +157,7 @@ func fetchTableDescriptors( return targets.EachTableID(func(id catid.DescID) error { tableDesc, err := descriptors.ByIDWithoutLeased(txn.KV()).WithoutNonPublic().Get().Table(ctx, id) if err != nil { - return err + return errors.Wrapf(err, "fetching table descriptor %d", id) } targetDescs = append(targetDescs, tableDesc) return nil diff --git a/pkg/ccl/changefeedccl/protected_timestamps_test.go b/pkg/ccl/changefeedccl/protected_timestamps_test.go index 9ac09309cac2..2c4e8152c032 100644 --- a/pkg/ccl/changefeedccl/protected_timestamps_test.go +++ b/pkg/ccl/changefeedccl/protected_timestamps_test.go @@ -25,7 +25,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigjob" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" @@ -451,16 +453,41 @@ func TestPTSRecordProtectsTargetsAndSystemTables(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - s, db, stopServer := startTestFullServer(t, feedTestOptions{}) + ctx := context.Background() + + // Useful for debugging. + require.NoError(t, log.SetVModule("spanconfigstore=2,store=2,reconciler=3,mvcc_gc_queue=2,kvaccessor=2")) + + settings := cluster.MakeTestingClusterSettings() + spanconfigjob.ReconciliationJobCheckpointInterval.Override(ctx, &settings.SV, 1*time.Second) + + // Keep track of where the spanconfig reconciler is up to. + lastReconcilerCheckpoint := atomic.Value{} + lastReconcilerCheckpoint.Store(hlc.Timestamp{}) + s, db, stopServer := startTestFullServer(t, feedTestOptions{ + knobsFn: func(knobs *base.TestingKnobs) { + if knobs.SpanConfig == nil { + knobs.SpanConfig = &spanconfig.TestingKnobs{} + } + scKnobs := knobs.SpanConfig.(*spanconfig.TestingKnobs) + scKnobs.JobOnCheckpointInterceptor = func(lastCheckpoint hlc.Timestamp) error { + now := hlc.Timestamp{WallTime: time.Now().UnixNano()} + t.Logf("reconciler checkpoint %s (%s)", lastCheckpoint, now.GoTime().Sub(lastCheckpoint.GoTime())) + lastReconcilerCheckpoint.Store(lastCheckpoint) + return nil + } + scKnobs.SQLWatcherCheckpointNoopsEveryDurationOverride = 1 * time.Second + }, + settings: settings, + }) + defer stopServer() execCfg := s.ExecutorConfig().(sql.ExecutorConfig) sqlDB := sqlutils.MakeSQLRunner(db) - sqlDB.Exec(t, `ALTER DATABASE system CONFIGURE ZONE USING gc.ttlseconds = 1`) sqlDB.Exec(t, "CREATE TABLE foo (a INT, b STRING)") sqlDB.Exec(t, `CREATE USER test`) sqlDB.Exec(t, `GRANT admin TO test`) ts := s.Clock().Now() - ctx := context.Background() fooDescr := cdctest.GetHydratedTableDescriptor(t, s.ExecutorConfig(), "d", "foo") var targets changefeedbase.Targets @@ -468,12 +495,30 @@ func TestPTSRecordProtectsTargetsAndSystemTables(t *testing.T) { TableID: fooDescr.GetID(), }) + // We need to give our PTS record a legit job ID so the protected ts + // reconciler doesn't delete it, so start up a dummy changefeed job and use its id. + registry := s.JobRegistry().(*jobs.Registry) + dummyJobDone := make(chan struct{}) + defer close(dummyJobDone) + registry.TestingWrapResumerConstructor(jobspb.TypeChangefeed, + func(raw jobs.Resumer) jobs.Resumer { + return &fakeResumer{done: dummyJobDone} + }) + var jobID jobspb.JobID + sqlDB.QueryRow(t, `CREATE CHANGEFEED FOR TABLE foo INTO 'null://'`).Scan(&jobID) + waitForJobStatus(sqlDB, t, jobID, `running`) + // Lay protected timestamp record. - ptr := createProtectedTimestampRecord(ctx, s.Codec(), 42, targets, ts) + ptr := createProtectedTimestampRecord(ctx, s.Codec(), jobID, targets, ts) require.NoError(t, execCfg.InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { return execCfg.ProtectedTimestampProvider.WithTxn(txn).Protect(ctx, ptr) })) + // Set GC TTL to a small value to make the tables GC'd. We need to set this + // *after* we set the PTS record so that we dont GC the tables before + // the PTS is applied/picked up. + sqlDB.Exec(t, `ALTER DATABASE system CONFIGURE ZONE USING gc.ttlseconds = 1`) + // The following code was shameless stolen from // TestShowTenantFingerprintsProtectsTimestamp which almost // surely copied it from the 2-3 other tests that have @@ -509,7 +554,7 @@ func TestPTSRecordProtectsTargetsAndSystemTables(t *testing.T) { var rangeID int64 row.Scan(&rangeID) refreshPTSReaderCache(s.Clock().Now(), tableName, databaseName) - t.Logf("enqueuing range %d for mvccGC", rangeID) + t.Logf("enqueuing range %d (table %s.%s) for mvccGC", rangeID, tableName, databaseName) sqlDB.Exec(t, `SELECT crdb_internal.kv_enqueue_replica($1, 'mvccGC', true)`, rangeID) } @@ -523,7 +568,21 @@ func TestPTSRecordProtectsTargetsAndSystemTables(t *testing.T) { // Change the user's password to update the users table. sqlDB.Exec(t, `ALTER USER test WITH PASSWORD 'testpass'`) + // Sleep for enough time to pass the configured GC threshold (1 second). time.Sleep(2 * time.Second) + + // Wait for the spanconfigs to be reconciled. + now := hlc.Timestamp{WallTime: time.Now().UnixNano()} + t.Logf("waiting for spanconfigs to be reconciled") + testutils.SucceedsWithin(t, func() error { + lastCheckpoint := lastReconcilerCheckpoint.Load().(hlc.Timestamp) + if lastCheckpoint.Less(now) { + return errors.Errorf("last checkpoint %s is not less than now %s", lastCheckpoint, now) + } + t.Logf("last reconciler checkpoint ok at %s", lastCheckpoint) + return nil + }, 1*time.Minute) + // If you want to GC all system tables: // // tabs := systemschema.MakeSystemTables() @@ -532,6 +591,7 @@ func TestPTSRecordProtectsTargetsAndSystemTables(t *testing.T) { // gcTestTableRange("system", t.GetName()) // } // } + t.Logf("GC'ing system tables") gcTestTableRange("system", "descriptor") gcTestTableRange("system", "zones") gcTestTableRange("system", "comments") diff --git a/pkg/spanconfig/spanconfigjob/job.go b/pkg/spanconfig/spanconfigjob/job.go index 7d48cbff1f20..90d642f97b31 100644 --- a/pkg/spanconfig/spanconfigjob/job.go +++ b/pkg/spanconfig/spanconfigjob/job.go @@ -30,7 +30,7 @@ type resumer struct { var _ jobs.Resumer = (*resumer)(nil) -var reconciliationJobCheckpointInterval = settings.RegisterDurationSetting( +var ReconciliationJobCheckpointInterval = settings.RegisterDurationSetting( settings.ApplicationLevel, "spanconfig.reconciliation_job.checkpoint_interval", "the frequency at which the span config reconciliation job checkpoints itself", @@ -104,17 +104,17 @@ func (r *resumer) Resume(ctx context.Context, execCtxI interface{}) (jobErr erro syncutil.Mutex util.EveryN }{} - persistCheckpointsMu.EveryN = util.Every(reconciliationJobCheckpointInterval.Get(settingValues)) + persistCheckpointsMu.EveryN = util.Every(ReconciliationJobCheckpointInterval.Get(settingValues)) - reconciliationJobCheckpointInterval.SetOnChange(settingValues, func(ctx context.Context) { + ReconciliationJobCheckpointInterval.SetOnChange(settingValues, func(ctx context.Context) { persistCheckpointsMu.Lock() defer persistCheckpointsMu.Unlock() - persistCheckpointsMu.EveryN = util.Every(reconciliationJobCheckpointInterval.Get(settingValues)) + persistCheckpointsMu.EveryN = util.Every(ReconciliationJobCheckpointInterval.Get(settingValues)) }) checkpointingDisabled := false shouldSkipRetry := false - var onCheckpointInterceptor func() error + var onCheckpointInterceptor func(lastCheckpoint hlc.Timestamp) error retryOpts := retry.Options{ InitialBackoff: 5 * time.Second, @@ -140,7 +140,7 @@ func (r *resumer) Resume(ctx context.Context, execCtxI interface{}) (jobErr erro started := timeutil.Now() if err := rc.Reconcile(ctx, lastCheckpoint, r.job.Session(), func() error { if onCheckpointInterceptor != nil { - if err := onCheckpointInterceptor(); err != nil { + if err := onCheckpointInterceptor(lastCheckpoint); err != nil { return err } } diff --git a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go index ca0b5a22a7c8..fc9e87f1f7ed 100644 --- a/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go +++ b/pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go @@ -163,6 +163,8 @@ func (k *KVAccessor) UpdateSpanConfigRecords( toUpsert []spanconfig.Record, minCommitTS, maxCommitTS hlc.Timestamp, ) error { + log.VInfof(ctx, 2, "kv accessor updating span configs: toDelete=%+v, toUpsert=%+v, minCommitTS=%s, maxCommitTS=%s", toDelete, toUpsert, minCommitTS, maxCommitTS) + if k.optionalTxn != nil { return k.updateSpanConfigRecordsWithTxn(ctx, toDelete, toUpsert, k.optionalTxn, minCommitTS, maxCommitTS) } diff --git a/pkg/spanconfig/spanconfigmanager/manager_test.go b/pkg/spanconfig/spanconfigmanager/manager_test.go index d9ee96394557..69986b235511 100644 --- a/pkg/spanconfig/spanconfigmanager/manager_test.go +++ b/pkg/spanconfig/spanconfigmanager/manager_test.go @@ -303,7 +303,7 @@ func TestReconciliationJobErrorAndRecovery(t *testing.T) { ManagerDisableJobCreation: true, // disable the automatic job creation JobDisableInternalRetry: true, SQLWatcherCheckpointNoopsEveryDurationOverride: 100 * time.Millisecond, - JobOnCheckpointInterceptor: func() error { + JobOnCheckpointInterceptor: func(_ hlc.Timestamp) error { mu.Lock() defer mu.Unlock() @@ -388,7 +388,7 @@ func TestReconciliationUsesRightCheckpoint(t *testing.T) { }, ManagerDisableJobCreation: true, // disable the automatic job creation SQLWatcherCheckpointNoopsEveryDurationOverride: 10 * time.Millisecond, - JobOnCheckpointInterceptor: func() error { + JobOnCheckpointInterceptor: func(_ hlc.Timestamp) error { select { case err := <-errCh: return err diff --git a/pkg/spanconfig/spanconfigreconciler/reconciler.go b/pkg/spanconfig/spanconfigreconciler/reconciler.go index 9e2c9d43ee98..122cd92fcaba 100644 --- a/pkg/spanconfig/spanconfigreconciler/reconciler.go +++ b/pkg/spanconfig/spanconfigreconciler/reconciler.go @@ -461,6 +461,10 @@ func updateSpanConfigRecords( } return err // not a retryable error, bubble up } + + if log.V(3) { + log.Infof(ctx, "successfully updated span config records: deleted = %+#v; upserted = %+#v", toDelete, toUpsert) + } return nil // we performed the update; we're done here } return nil diff --git a/pkg/spanconfig/spanconfigstore/store.go b/pkg/spanconfig/spanconfigstore/store.go index 20b78331473d..922b702d0c99 100644 --- a/pkg/spanconfig/spanconfigstore/store.go +++ b/pkg/spanconfig/spanconfigstore/store.go @@ -360,7 +360,7 @@ func (s *Store) maybeLogUpdate(ctx context.Context, update *spanconfig.Update) e // Log if there is a SpanConfig change in any field other than // ProtectedTimestamps to avoid logging PTS updates. - if found && curSpanConfig.HasConfigurationChange(nextSC) { + if log.V(2) || (found && curSpanConfig.HasConfigurationChange(nextSC)) { log.KvDistribution.Infof(ctx, "changing the spanconfig for span:%+v from:%+v to:%+v", target, curSpanConfig, nextSC) diff --git a/pkg/spanconfig/testing_knobs.go b/pkg/spanconfig/testing_knobs.go index 2226c9ecb20d..ad968d3a6bed 100644 --- a/pkg/spanconfig/testing_knobs.go +++ b/pkg/spanconfig/testing_knobs.go @@ -49,7 +49,7 @@ type TestingKnobs struct { // JobPersistCheckpointInterceptor, if set, is invoked before the // reconciliation job persists checkpoints. - JobOnCheckpointInterceptor func() error + JobOnCheckpointInterceptor func(lastCheckpoint hlc.Timestamp) error // KVSubscriberRangeFeedKnobs control lifecycle events for the rangefeed // underlying the KVSubscriber. From a785188f848fcdd0ba8f9d2cc41c038df77137a9 Mon Sep 17 00:00:00 2001 From: Ibrahim Kettaneh Date: Tue, 7 Jan 2025 09:47:42 -0500 Subject: [PATCH 096/126] raft: avoid allocating supportMap for every LeadSupportUntil calculation This adds supportExpMap that hangs off the fortificationTracker. This is useful to avoid allocating the map for every LeadSupportUntil calculation. This is now possible since all the calls to ComputeLeadSupportUntil are done with a lock held. Note that this doesn't seem to improve the performance of ComputeLeadSupportUntil when the number of voters are low. The reason is that there is a go compiler optimization that seems to make an on-stack map allocation if the size is small. References: #137264 Release note: None --- pkg/raft/status.go | 1 - pkg/raft/tracker/fortificationtracker.go | 15 ++++--- pkg/raft/tracker/fortificationtracker_test.go | 39 +++++++++++++++++++ 3 files changed, 49 insertions(+), 6 deletions(-) diff --git a/pkg/raft/status.go b/pkg/raft/status.go index 4dbe9cb2a645..66a949d389da 100644 --- a/pkg/raft/status.go +++ b/pkg/raft/status.go @@ -98,7 +98,6 @@ func getBasicStatus(r *raft) BasicStatus { // NOTE: we assign to LeadSupportUntil even if RaftState is not currently // StateLeader. The replica may have been the leader and stepped down to a // follower before its lead support ran out. - //s.LeadSupportUntil = hlc.Timestamp{} s.LeadSupportUntil = r.fortificationTracker.LeadSupportUntil(r.state) assertTrue((s.RaftState == pb.StateLeader) == (s.Lead == r.id), "inconsistent lead / raft state") diff --git a/pkg/raft/tracker/fortificationtracker.go b/pkg/raft/tracker/fortificationtracker.go index 80aba5e6e637..8e66c3eabe48 100644 --- a/pkg/raft/tracker/fortificationtracker.go +++ b/pkg/raft/tracker/fortificationtracker.go @@ -100,6 +100,12 @@ type FortificationTracker struct { // LeadSupportUntil is called by every request trying to evaluate the lease's // status. computedLeadSupportUntil hlc.Timestamp + + // supportExpMap is a map that hangs off the fortificationTracker to prevent + // allocations on every call to ComputeLeadSupportUntil. It stores the + // SupportFrom expiration timestamps for the voters, which are then used to + // calculate the LeadSupportUntil. + supportExpMap map[pb.PeerID]hlc.Timestamp } // NewFortificationTracker initializes a FortificationTracker. @@ -112,6 +118,7 @@ func NewFortificationTracker( fortification: map[pb.PeerID]pb.Epoch{}, votersSupport: map[pb.PeerID]bool{}, logger: logger, + supportExpMap: map[pb.PeerID]hlc.Timestamp{}, } return &st } @@ -211,9 +218,7 @@ func (ft *FortificationTracker) ComputeLeadSupportUntil(state pb.StateType) hlc. return ft.computedLeadSupportUntil // fast-path for no fortification } - // TODO(ibrahim): avoid this map allocation as we're calling LeadSupportUntil - // on every tick, on every new fortification, and on config changes. - supportExpMap := make(map[pb.PeerID]hlc.Timestamp) + clear(ft.supportExpMap) ft.config.Voters.Visit(func(id pb.PeerID) { if supportEpoch, ok := ft.fortification[id]; ok { curEpoch, curExp := ft.storeLiveness.SupportFrom(id) @@ -223,11 +228,11 @@ func (ft *FortificationTracker) ComputeLeadSupportUntil(state pb.StateType) hlc. // supporting the leader's store at the epoch in the MsgFortifyLeaderResp // message. if curEpoch == supportEpoch { - supportExpMap[id] = curExp + ft.supportExpMap[id] = curExp } } }) - ft.computedLeadSupportUntil = ft.config.Voters.LeadSupportExpiration(supportExpMap) + ft.computedLeadSupportUntil = ft.config.Voters.LeadSupportExpiration(ft.supportExpMap) return ft.computedLeadSupportUntil } diff --git a/pkg/raft/tracker/fortificationtracker_test.go b/pkg/raft/tracker/fortificationtracker_test.go index 4c8094542aec..1ee544d663bf 100644 --- a/pkg/raft/tracker/fortificationtracker_test.go +++ b/pkg/raft/tracker/fortificationtracker_test.go @@ -6,6 +6,7 @@ package tracker import ( + "fmt" "testing" "github.com/cockroachdb/cockroach/pkg/raft/quorum" @@ -905,6 +906,44 @@ func TestConfigChangeSafe(t *testing.T) { } } +// BenchmarkComputeLeadSupportUntil keeps calling ComputeLeadSupportUntil() for +// different number of members. +func BenchmarkComputeLeadSupportUntil(b *testing.B) { + ts := func(ts int64) hlc.Timestamp { + return hlc.Timestamp{ + WallTime: ts, + } + } + + for _, members := range []int{1, 3, 5, 7, 100} { + b.Run(fmt.Sprintf("members=%d", members), func(b *testing.B) { + // Prepare the mock store liveness, and record fortifications. + livenessMap := map[pb.PeerID]mockLivenessEntry{} + for i := 1; i <= members; i++ { + livenessMap[pb.PeerID(i)] = makeMockLivenessEntry(10, ts(100)) + } + + mockLiveness := makeMockStoreLiveness(livenessMap) + cfg := quorum.MakeEmptyConfig() + for i := 1; i <= members; i++ { + cfg.Voters[0][pb.PeerID(i)] = struct{}{} + } + + ft := NewFortificationTracker(&cfg, mockLiveness, raftlogger.DiscardLogger) + for i := 1; i <= members; i++ { + ft.RecordFortification(pb.PeerID(i), 10) + } + + // The benchmark actually starts here. + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + ft.ComputeLeadSupportUntil(pb.StateLeader) + } + }) + } +} + type mockLivenessEntry struct { epoch pb.Epoch ts hlc.Timestamp From d4d9811d84325062306057ba4a20293be9e27a37 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 14 Jan 2025 17:45:22 +0100 Subject: [PATCH 097/126] DEPS: fix build This broke silently in https://github.com/cockroachdb/cockroach/pull/138283. --- build/patches/com_github_buchgr_bazel_remote.patch | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/build/patches/com_github_buchgr_bazel_remote.patch b/build/patches/com_github_buchgr_bazel_remote.patch index 5bf4f52a070a..7bc6240bc94b 100644 --- a/build/patches/com_github_buchgr_bazel_remote.patch +++ b/build/patches/com_github_buchgr_bazel_remote.patch @@ -153,9 +153,9 @@ index f60cf80..ba707d3 100644 - "@go_googleapis//google/bytestream:bytestream_go_proto", - "@go_googleapis//google/rpc:code_go_proto", - "@go_googleapis//google/rpc:status_go_proto", -+ "@org_golang_google_genproto//googleapis/bytestream:go_default_library", -+ "@org_golang_google_genproto//googleapis/rpc/code:go_default_library", -+ "@org_golang_google_genproto//googleapis/rpc/status:go_default_library", ++ "@org_golang_google_genproto_googleapis_bytestream//:go_default_library", ++ "@org_golang_google_genproto_googleapis_rpc//code:go_default_library", ++ "@org_golang_google_genproto_googleapis_rpc//status:go_default_library", "@org_golang_google_grpc//:go_default_library", "@org_golang_google_grpc//codes:go_default_library", "@org_golang_google_grpc//encoding/gzip:go_default_library", @@ -164,7 +164,7 @@ index f60cf80..ba707d3 100644 "@com_github_golang_protobuf//proto:go_default_library", "@com_github_google_uuid//:go_default_library", - "@go_googleapis//google/bytestream:bytestream_go_proto", -+ "@org_golang_google_genproto//googleapis/bytestream:go_default_library", ++ "@org_golang_google_genproto_googleapis_bytestream//:go_default_library", "@org_golang_google_grpc//:go_default_library", "@org_golang_google_grpc//codes:go_default_library", "@org_golang_google_grpc//status:go_default_library", From 3aa2724607247352009e73b6b79a7f7ddea69110 Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Tue, 14 Jan 2025 10:59:59 -0600 Subject: [PATCH 098/126] build: update `rules_go` To pull in cockroachdb/rules_go#24 Epic: CRDB-41952 Release note: None --- WORKSPACE | 8 ++++---- build/bazelutil/distdir_files.bzl | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/WORKSPACE b/WORKSPACE index 745384c5badf..425dbf4de2af 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -8,12 +8,12 @@ load("@bazel_tools//tools/build_defs/repo:http.bzl", "http_archive") # Load go bazel tools. This gives us access to the go bazel SDK/toolchains. http_archive( name = "io_bazel_rules_go", - sha256 = "094b2d9b480eb508bd352914a085c5901d0f3c4520833a12b9197394a80619c7", - strip_prefix = "cockroachdb-rules_go-548c13b", + sha256 = "fc43f42615292c9c3ca6dc211b19ce45986562b0efbca3635e83a70eb011e293", + strip_prefix = "cockroachdb-rules_go-59db5f5", urls = [ - # cockroachdb/rules_go as of 548c13bf30e840418237a12cb5fbd0be391278bd + # cockroachdb/rules_go as of 59db5f5a4a65d6671dd62dd4aea42757b09a1fa3 # (upstream release-0.50 plus a few patches). - "https://storage.googleapis.com/public-bazel-artifacts/bazel/cockroachdb-rules_go-v0.27.0-529-g548c13b.tar.gz", + "https://storage.googleapis.com/public-bazel-artifacts/bazel/cockroachdb-rules_go-v0.27.0-530-g59db5f5.tar.gz", ], ) diff --git a/build/bazelutil/distdir_files.bzl b/build/bazelutil/distdir_files.bzl index ba24dc4d7d19..cfb8e7f787ba 100644 --- a/build/bazelutil/distdir_files.bzl +++ b/build/bazelutil/distdir_files.bzl @@ -1207,7 +1207,7 @@ DISTDIR_FILES = { "https://storage.googleapis.com/public-bazel-artifacts/bazel/bmatcuk-doublestar-v4.0.1-0-gf7a8118.tar.gz": "d11c3b3a45574f89d6a6b2f50e53feea50df60407b35f36193bf5815d32c79d1", "https://storage.googleapis.com/public-bazel-artifacts/bazel/cockroachdb-protobuf-3f5d91f.tar.gz": "6d4e7fe1cbd958dee69ce9becbf8892d567f082b6782d3973a118d0aa00807a8", "https://storage.googleapis.com/public-bazel-artifacts/bazel/cockroachdb-rules_foreign_cc-8d34d77.tar.gz": "03afebfc3f173666a3820a29512265c710c3a08d0082ba77469779d3e3af5a11", - "https://storage.googleapis.com/public-bazel-artifacts/bazel/cockroachdb-rules_go-v0.27.0-529-g548c13b.tar.gz": "094b2d9b480eb508bd352914a085c5901d0f3c4520833a12b9197394a80619c7", + "https://storage.googleapis.com/public-bazel-artifacts/bazel/cockroachdb-rules_go-v0.27.0-530-g59db5f5.tar.gz": "fc43f42615292c9c3ca6dc211b19ce45986562b0efbca3635e83a70eb011e293", "https://storage.googleapis.com/public-bazel-artifacts/bazel/google-starlark-go-e043a3d.tar.gz": "a35c6468e0e0921833a63290161ff903295eaaf5915200bbce272cbc8dfd1c1c", "https://storage.googleapis.com/public-bazel-artifacts/bazel/googleapis-83c3605afb5a39952bf0a0809875d41cf2a558ca.zip": "ba694861340e792fd31cb77274eacaf6e4ca8bda97707898f41d8bebfd8a4984", "https://storage.googleapis.com/public-bazel-artifacts/bazel/platforms-0.0.10.tar.gz": "218efe8ee736d26a3572663b374a253c012b716d8af0c07e842e82f238a0a7ee", From cc8e09034ac80a91481104dd307187fc000ff643 Mon Sep 17 00:00:00 2001 From: DarrylWong Date: Tue, 14 Jan 2025 11:52:35 -0500 Subject: [PATCH 099/126] roachtest: fix single node sysbench setup The single node setup was accidentally using HAProxy for the prepare step, causing it to silently fail and segfault later on. Since segfaults are an ignored error, the single node tests were silently failing. --- pkg/cmd/roachtest/tests/sysbench.go | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/pkg/cmd/roachtest/tests/sysbench.go b/pkg/cmd/roachtest/tests/sysbench.go index 57de742da28a..ea0b0f14675a 100644 --- a/pkg/cmd/roachtest/tests/sysbench.go +++ b/pkg/cmd/roachtest/tests/sysbench.go @@ -185,12 +185,21 @@ func runSysbench(ctx context.Context, t test.Test, c cluster.Cluster, opts sysbe var start time.Time runWorkload := func(ctx context.Context) error { t.Status("preparing workload") - c.Run(ctx, option.WithNodes(c.WorkloadNode()), opts.cmd(true /* haproxy */)+" prepare") + cmd := opts.cmd(useHAProxy /* haproxy */) + result, err := c.RunWithDetailsSingleNode(ctx, t.L(), option.WithNodes(c.WorkloadNode()), cmd+" prepare") + if err != nil { + return err + } else if strings.Contains(result.Stdout, "FATAL") { + // sysbench prepare doesn't exit on errors for some reason, so we have + // to check that it didn't silently fail. We've seen it do so, causing + // the run step to segfault. Segfaults are an ignored error, so in the + // past, this would cause the test to silently fail. + return errors.Newf("sysbench prepare failed with FATAL error") + } t.Status("running workload") - cmd := opts.cmd(useHAProxy /* haproxy */) + " run" start = timeutil.Now() - result, err := c.RunWithDetailsSingleNode(ctx, t.L(), option.WithNodes(c.WorkloadNode()), cmd) + result, err = c.RunWithDetailsSingleNode(ctx, t.L(), option.WithNodes(c.WorkloadNode()), cmd+" run") // Sysbench occasionally segfaults. When that happens, don't fail the // test. From 271363e9b30b5dd226ebb7acadb0298dd3d0c1c2 Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Tue, 14 Jan 2025 11:31:33 -0500 Subject: [PATCH 100/126] roachtest: disable shared process rebalance/by-load roachtests In #129117, shared process deployments were added to the `rebalance/by-load/*/mixed-version` roachtests. Despite multiple deflaking attempts (#131787, #133681, #136115, #136116), these continue to fail weekly. The root cause is currently unknown. Part of: #139037 Release note: None --- pkg/cmd/roachtest/tests/rebalance_load.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pkg/cmd/roachtest/tests/rebalance_load.go b/pkg/cmd/roachtest/tests/rebalance_load.go index 5dd16d324495..6c0c44f5c474 100644 --- a/pkg/cmd/roachtest/tests/rebalance_load.go +++ b/pkg/cmd/roachtest/tests/rebalance_load.go @@ -96,6 +96,12 @@ func registerRebalanceLoad(r registry.Registry) { ), // Only use the latest version of each release to work around #127029. mixedversion.AlwaysUseLatestPredecessors, + // TODO(kvoli): Re-enable shared process deployments for mixed version + // variant #139037. + mixedversion.EnabledDeploymentModes( + mixedversion.SystemOnlyDeployment, + mixedversion.SeparateProcessDeployment, + ), ) mvt.OnStartup("maybe enable split/scatter on tenant", func(ctx context.Context, l *logger.Logger, r *rand.Rand, h *mixedversion.Helper) error { From aa2769a16a965a7d5e806fb852ae1f85c133e975 Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Thu, 19 Dec 2024 15:41:17 -0800 Subject: [PATCH 101/126] sql/row: use Put instead of CPut when updating value of secondary index When an UPDATE statement changes the value but not the key of a secondary index (e.g. an update to the stored columns of a secondary index) we need to write a new version of the secondary index KV with the new value. We were using a CPutAllowingIfNotExists to do this, which verified that if the KV existed, the expected value was the value before update. But there's no need for this verification. We have other mechanisms to detect a write-write conflict with any other transaction that could have changed the value concurrently. We can simply use a Put to overwrite the previous value. This also matches what we do for the primary index when the PK doesn't change. Epic: None Release note: None --- .../testdata/secondary_index_column_families | 8 ++++---- pkg/sql/row/updater.go | 16 ++++++++++------ 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families b/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families index 5a9707457255..1844cc6bdaad 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families +++ b/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families @@ -228,10 +228,10 @@ SELECT y FROM t@i WHERE x IS NULL Scan /Table/107/2/{NULL-!NULL} # Ensure that updates only touch the changed column families. -query T kvtrace(CPut,prefix=/Table/107/2/) +query T kvtrace(Put,prefix=/Table/107/2/) UPDATE t SET y = 5 WHERE x = 1 ---- -CPut /Table/107/2/1/1/1 -> /TUPLE/2:2:Int/5 (replacing /TUPLE/2:2:Int/2, if exists) +Put /Table/107/2/1/1/1 -> /TUPLE/2:2:Int/5 # Test composite datatypes. statement ok @@ -404,7 +404,7 @@ CPut /Table/112/3/2/5/1 -> /TUPLE/8:8:Int/8 query T kvtrace(Put,Del,CPut,prefix=/Table/112/2/) UPDATE t SET b = 4, c = NULL, d = NULL, e = 7, f = NULL WHERE y = 2 ---- -CPut /Table/112/2/2/1/2/1 -> /TUPLE/3:3:Int/3/1:4:Int/4 (replacing /TUPLE/3:3:Int/3, if exists) +Put /Table/112/2/2/1/2/1 -> /TUPLE/3:3:Int/3/1:4:Int/4 Del /Table/112/2/2/1/3/1 CPut /Table/112/2/2/1/4/1 -> /TUPLE/7:7:Int/7 (expecting does not exist) Del /Table/112/2/2/1/5/1 @@ -495,4 +495,4 @@ CPut /Table/113/2/5/1/0 -> /BYTES/0x33061308 (expecting does not exist) query T kvtrace(Put,Del,CPut,prefix=/Table/113/2/) UPDATE t SET z = 5 where y = 5 ---- -CPut /Table/113/2/5/1/0 -> /BYTES/0x330a1308 (replacing /BYTES/0x33061308, if exists) +Put /Table/113/2/5/1/0 -> /BYTES/0x330a1308 diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index 55cb1e782c0b..29c2c0770639 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -398,18 +398,18 @@ func (ru *Updater) UpdateRow( oldEntry, newEntry := &oldEntries[oldIdx], &newEntries[newIdx] if oldEntry.Family == newEntry.Family { // If the families are equal, then check if the keys have changed. If so, delete the old key. - // Then, issue a CPut for the new value of the key if the value has changed. + // Then, issue a CPut for the new key or a Put if only the value has changed. // Because the indexes will always have a k/v for family 0, it suffices to only // add foreign key checks in this case, because we are guaranteed to enter here. oldIdx++ newIdx++ - var expValue []byte + var sameKey bool if !bytes.Equal(oldEntry.Key, newEntry.Key) { if err := ru.Helper.deleteIndexEntry(ctx, batch, index, ru.Helper.secIndexValDirs[i], oldEntry, traceKV); err != nil { return nil, err } } else if !newEntry.Value.EqualTagAndData(oldEntry.Value) { - expValue = oldEntry.Value.TagAndDataBytes() + sameKey = true } else if !index.IsTemporaryIndexForBackfill() { // If this is a temporary index for backfill, we want to make sure we write out all // index values even in the case where they should be the same. We do this because the @@ -428,13 +428,17 @@ func (ru *Updater) UpdateRow( if traceKV { k := keys.PrettyPrint(ru.Helper.secIndexValDirs[i], newEntry.Key) v := newEntry.Value.PrettyPrint() - if expValue != nil { - log.VEventf(ctx, 2, "CPut %s -> %v (replacing %v, if exists)", k, v, oldEntry.Value.PrettyPrint()) + if sameKey { + log.VEventf(ctx, 2, "Put %s -> %v", k, v) } else { log.VEventf(ctx, 2, "CPut %s -> %v (expecting does not exist)", k, v) } } - batch.CPutAllowingIfNotExists(newEntry.Key, &newEntry.Value, expValue) + if sameKey { + batch.Put(newEntry.Key, &newEntry.Value) + } else { + batch.CPutAllowingIfNotExists(newEntry.Key, &newEntry.Value, nil /* expValue */) + } } writtenIndexes.Add(i) } else if oldEntry.Family < newEntry.Family { From 003a622fa46dd9e1dd81a2a98cd7393873ca33b0 Mon Sep 17 00:00:00 2001 From: Ibrahim Kettaneh Date: Mon, 6 Jan 2025 17:56:02 -0500 Subject: [PATCH 102/126] raft: remove atomicTimestamp for fortificationtracker This commit removes the struct atomicTimestamp and replaces it with hlc.Timestamp directly without locks. This is now possible as we only update LeadSupportUntil every tick, and it won't change on every status call, which are called while we hold the read lock. References: #137264 Release note: None --- pkg/raft/raft.go | 2 + pkg/raft/raft_test.go | 80 +++++- pkg/raft/rawnode_test.go | 45 ++- pkg/raft/status.go | 2 +- .../confchange_v2_add_double_auto.txt | 266 ++++++++++-------- pkg/raft/tracker/BUILD.bazel | 1 - pkg/raft/tracker/fortificationtracker.go | 85 ++---- pkg/raft/tracker/fortificationtracker_test.go | 45 +-- 8 files changed, 309 insertions(+), 217 deletions(-) diff --git a/pkg/raft/raft.go b/pkg/raft/raft.go index f1bcdf60b1c2..83f1dec0af1e 100644 --- a/pkg/raft/raft.go +++ b/pkg/raft/raft.go @@ -923,6 +923,7 @@ func (r *raft) bcastDeFortify() { assertTrue(r.fortificationTracker.CanDefortify(), "unsafe to de-fortify") r.trk.Visit(func(id pb.PeerID, _ *tracker.Progress) { + fmt.Printf("Sending defortify to %v\n", id) r.sendDeFortify(id) }) } @@ -1174,6 +1175,7 @@ func (r *raft) tickElection() { r.heartbeatElapsed++ if r.heartbeatElapsed >= r.heartbeatTimeout { r.heartbeatElapsed = 0 + if r.shouldBcastDeFortify() { r.bcastDeFortify() } diff --git a/pkg/raft/raft_test.go b/pkg/raft/raft_test.go index 62022f06dd5a..ddb3522e8fae 100644 --- a/pkg/raft/raft_test.go +++ b/pkg/raft/raft_test.go @@ -3590,7 +3590,34 @@ func TestLeaderTransferToUpToDateNodeFromFollower(t *testing.T) { // transfer target, even before (and regardless of if) the target receives the // MsgTimeoutNow and campaigns. func TestLeaderTransferLeaderStepsDownImmediately(t *testing.T) { - nt := newNetwork(nil, nil, nil) + testutils.RunTrueAndFalse(t, "store-liveness-enabled", + func(t *testing.T, storeLivenessEnabled bool) { + testLeaderTransferLeaderStepsDownImmediately(t, storeLivenessEnabled) + }) +} + +func testLeaderTransferLeaderStepsDownImmediately(t *testing.T, storeLivenessEnabled bool) { + var fabric *raftstoreliveness.LivenessFabric + var n1, n2, n3 *raft + + if storeLivenessEnabled { + fabric = raftstoreliveness.NewLivenessFabricWithPeers(1, 2, 3) + n1 = newTestRaft(1, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(fabric.GetStoreLiveness(1))) + n2 = newTestRaft(2, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(fabric.GetStoreLiveness(2))) + n3 = newTestRaft(3, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(fabric.GetStoreLiveness(3))) + } else { + n1 = newTestRaft(1, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(raftstoreliveness.Disabled{})) + n2 = newTestRaft(2, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(raftstoreliveness.Disabled{})) + n3 = newTestRaft(3, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(raftstoreliveness.Disabled{})) + } + + nt := newNetworkWithConfigAndLivenessFabric(nil, fabric, n1, n2, n3) nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) // Isolate node 3. It is up-to-date, so the leadership transfer will be @@ -3609,6 +3636,15 @@ func TestLeaderTransferLeaderStepsDownImmediately(t *testing.T) { require.Equal(t, uint64(1), lead.Term) checkLeaderTransferState(t, lead, pb.StateFollower, None) + // With leader leases, the ex-leader would send a MsgDefortifyLeader to + // its followers when the support is expired. + if storeLivenessEnabled { + nt.livenessFabric.SetSupportExpired(1, true) + lead.tick() + nt.send(lead.readMessages()...) + nt.livenessFabric.SetSupportExpired(1, false) + } + // Eventually, the previous leader gives up on waiting and calls an election // to reestablish leadership at the next term. for i := int64(0); i < lead.randomizedElectionTimeout; i++ { @@ -4057,23 +4093,57 @@ func TestLeaderTransferDifferentTerms(t *testing.T) { // stale follower (a follower still at an earlier term) will cause the follower // to call an election which it can not win. func TestLeaderTransferStaleFollower(t *testing.T) { - nt := newNetwork(nil, nil, nil) + testutils.RunTrueAndFalse(t, "store-liveness-enabled", + func(t *testing.T, storeLivenessEnabled bool) { + testLeaderTransferStaleFollower(t, storeLivenessEnabled) + }) +} + +func testLeaderTransferStaleFollower(t *testing.T, storeLivenessEnabled bool) { + var fabric *raftstoreliveness.LivenessFabric + var n1, n2, n3 *raft + + if storeLivenessEnabled { + fabric = raftstoreliveness.NewLivenessFabricWithPeers(1, 2, 3) + n1 = newTestRaft(1, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(fabric.GetStoreLiveness(1))) + n2 = newTestRaft(2, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(fabric.GetStoreLiveness(2))) + n3 = newTestRaft(3, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(fabric.GetStoreLiveness(3))) + } else { + n1 = newTestRaft(1, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(raftstoreliveness.Disabled{})) + n2 = newTestRaft(2, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(raftstoreliveness.Disabled{})) + n3 = newTestRaft(3, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)), + withStoreLiveness(raftstoreliveness.Disabled{})) + } + + nt := newNetworkWithConfigAndLivenessFabric(nil, fabric, n1, n2, n3) nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup}) - n1 := nt.peers[1].(*raft) - n2 := nt.peers[2].(*raft) - n3 := nt.peers[3].(*raft) nodes := []*raft{n1, n2, n3} // Attempt to transfer leadership to node 3. The MsgTimeoutNow is sent // immediately and node 1 steps down as leader, but node 3 does not receive // the message due to a network partition. nt.isolate(3) + nt.send(pb.Message{From: 3, To: 1, Type: pb.MsgTransferLeader}) for _, n := range nodes { require.Equal(t, pb.StateFollower, n.state) require.Equal(t, uint64(1), n.Term) } + // With leader leases, the ex-leader would send a MsgDefortifyLeader to + // its followers when the support is expired. + if storeLivenessEnabled { + nt.livenessFabric.SetSupportExpired(1, true) + n1.tick() + nt.send(nt.filter(n1.readMessages())...) + nt.livenessFabric.SetSupportExpired(1, false) + } + // Eventually, the previous leader gives up on waiting and calls an election // to reestablish leadership at the next term. Node 3 does not hear about this // either. diff --git a/pkg/raft/rawnode_test.go b/pkg/raft/rawnode_test.go index 70502d49eea1..2871a17664b8 100644 --- a/pkg/raft/rawnode_test.go +++ b/pkg/raft/rawnode_test.go @@ -68,6 +68,12 @@ func TestRawNodeStep(t *testing.T) { // that it applies and that the resulting ConfState matches expectations, and for // joint configurations makes sure that they are exited successfully. func TestRawNodeProposeAndConfChange(t *testing.T) { + testutils.RunTrueAndFalse(t, "store-liveness-enabled", + func(t *testing.T, storeLivenessEnabled bool) { + testRawNodeProposeAndConfChange(t, storeLivenessEnabled) + }) +} +func testRawNodeProposeAndConfChange(t *testing.T, storeLivenessEnabled bool) { testCases := []struct { cc pb.ConfChangeI exp pb.ConfState @@ -180,10 +186,28 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { for _, tc := range testCases { t.Run("", func(t *testing.T) { s := newTestMemoryStorage(withPeers(1)) - rawNode, err := NewRawNode(newTestConfig(1, 10, 1, s)) - require.NoError(t, err) + var fabric *raftstoreliveness.LivenessFabric + var rawNode *RawNode + var err error + if storeLivenessEnabled { + fabric = raftstoreliveness.NewLivenessFabricWithPeers(1, 2, 3) + rawNode, err = NewRawNode(newTestConfig(1, 10, 1, s, + withStoreLiveness(fabric.GetStoreLiveness(1)))) + } else { + rawNode, err = NewRawNode(newTestConfig(1, 10, 1, s, + withStoreLiveness(raftstoreliveness.Disabled{}))) + } + require.NoError(t, err) rawNode.Campaign() + + if storeLivenessEnabled { + // This is a bit of a hack: we need to make sure that the leader doesn't + // return a leaderMaxSupported to be an infinite time. Since we won't be + // able to perform a conf change until the current time is past the + // leaderMaxSupported. + fabric.WithdrawSupportForPeerFromAllPeers(1) + } proposed := false var ( lastIndex uint64 @@ -211,6 +235,11 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { } } rawNode.Advance(rd) + if storeLivenessEnabled { + // Revert the support state to how it was so that the test can run + // without having peer 1 not supported. + fabric.GrantSupportForPeerFromAllPeers(1) + } // Once we are the leader, propose a command and a ConfChange. if !proposed && rd.HardState.Lead == rawNode.raft.id { require.NoError(t, rawNode.Propose([]byte("somedata"))) @@ -353,8 +382,20 @@ func testRawNodeJointAutoLeave(t *testing.T, storeLivenessEnabled bool) { if cc != nil { // Force it to step down. rawNode.Step(pb.Message{Type: pb.MsgHeartbeatResp, From: 1, Term: rawNode.raft.Term + 1}) + + if storeLivenessEnabled { + // At this point, the leader is attempting to step down, and it will + // need to wait until the support has expired. + fabric.SetSupportExpired(rawNode.raft.id, true) + rawNode.Tick() + } + require.Equal(t, pb.StateFollower, rawNode.raft.state) if storeLivenessEnabled { + // We can now restore the support so that the test can proceed as + // expected. + fabric.SetSupportExpired(rawNode.raft.id, false) + // And also wait for defortification. for range rawNode.raft.heartbeatTimeout { rawNode.Tick() diff --git a/pkg/raft/status.go b/pkg/raft/status.go index 66a949d389da..851fa6244093 100644 --- a/pkg/raft/status.go +++ b/pkg/raft/status.go @@ -98,7 +98,7 @@ func getBasicStatus(r *raft) BasicStatus { // NOTE: we assign to LeadSupportUntil even if RaftState is not currently // StateLeader. The replica may have been the leader and stepped down to a // follower before its lead support ran out. - s.LeadSupportUntil = r.fortificationTracker.LeadSupportUntil(r.state) + s.LeadSupportUntil = r.fortificationTracker.LeadSupportUntil() assertTrue((s.RaftState == pb.StateLeader) == (s.Lead == r.id), "inconsistent lead / raft state") return s diff --git a/pkg/raft/testdata/confchange_v2_add_double_auto.txt b/pkg/raft/testdata/confchange_v2_add_double_auto.txt index f4674e89df2a..fe8c0e4ebb67 100644 --- a/pkg/raft/testdata/confchange_v2_add_double_auto.txt +++ b/pkg/raft/testdata/confchange_v2_add_double_auto.txt @@ -63,6 +63,7 @@ CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 v3 INFO 1 switched to configuration voters=(1 2 3)&&(1) autoleave +INFO 1 ignoring conf change {ConfChangeTransitionAuto [] []} at config voters=(1 2 3)&&(1) autoleave: lead support has not caught up to previous configuration INFO initiating automatic transition out of joint configuration voters=(1 2 3)&&(1) autoleave # n1 immediately probes n2 and n3. @@ -71,7 +72,7 @@ stabilize 1 > 1 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" Messages: 1->2 MsgFortifyLeader Term:1 Log:0/0 1->3 MsgFortifyLeader Term:1 Log:0/0 @@ -110,14 +111,14 @@ stabilize 1 2 Messages: 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ 1/4 EntryConfChangeV2 v2 v3 - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" ] 1->2 MsgSnap Term:1 Log:0/0 Snapshot: Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true > 2 receiving messages 1->2 MsgApp Term:1 Log:1/3 Commit:4 Entries:[ 1/4 EntryConfChangeV2 v2 v3 - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" ] DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 1->2 MsgSnap Term:1 Log:0/0 @@ -141,13 +142,13 @@ stabilize 1 2 > 1 handling Ready Ready MustSync=false: Messages: - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/4 Commit:4 Entries:[1/5 EntryNormal ""] > 2 handling Ready Ready MustSync=true: Entries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" Messages: 2->1 MsgAppResp Term:1 Log:0/5 Commit:4 > 1 receiving messages @@ -156,26 +157,61 @@ stabilize 1 2 Ready MustSync=true: HardState Term:1 Vote:1 Commit:5 Lead:1 LeadEpoch:1 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" Messages: 1->2 MsgApp Term:1 Log:1/5 Commit:5 - INFO 1 switched to configuration voters=(1 2 3) + INFO initiating automatic transition out of joint configuration voters=(1 2 3)&&(1) autoleave > 2 receiving messages 1->2 MsgApp Term:1 Log:1/5 Commit:5 > 1 handling Ready - Ready MustSync=false: + Ready MustSync=true: + Entries: + 1/6 EntryConfChangeV2 Messages: - 1->3 MsgFortifyLeader Term:1 Log:0/0 + 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] > 2 handling Ready Ready MustSync=true: HardState Term:1 Commit:5 Lead:1 LeadEpoch:1 CommittedEntries: - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" Messages: 2->1 MsgAppResp Term:1 Log:0/5 Commit:5 - INFO 2 switched to configuration voters=(1 2 3) > 1 receiving messages 2->1 MsgAppResp Term:1 Log:0/5 Commit:5 +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2] +> 2 handling Ready + Ready MustSync=true: + Entries: + 1/6 EntryConfChangeV2 + Messages: + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 +> 1 handling Ready + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:6 Lead:1 LeadEpoch:1 + CommittedEntries: + 1/6 EntryConfChangeV2 + Messages: + 1->2 MsgApp Term:1 Log:1/6 Commit:6 + INFO 1 switched to configuration voters=(1 2 3) +> 2 receiving messages + 1->2 MsgApp Term:1 Log:1/6 Commit:6 +> 1 handling Ready + Ready MustSync=false: + Messages: + 1->3 MsgFortifyLeader Term:1 Log:0/0 +> 2 handling Ready + Ready MustSync=true: + HardState Term:1 Commit:6 Lead:1 LeadEpoch:1 + CommittedEntries: + 1/6 EntryConfChangeV2 + Messages: + 2->1 MsgAppResp Term:1 Log:0/6 Commit:6 + INFO 2 switched to configuration voters=(1 2 3) +> 1 receiving messages + 2->1 MsgAppResp Term:1 Log:0/6 Commit:6 # n3 immediately receives a snapshot in the final configuration. stabilize 1 3 @@ -200,42 +236,44 @@ stabilize 1 3 3->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) DEBUG 1 received MsgAppResp(rejected, hint: (index 0, term 0)) from 3 for index 3 DEBUG 1 decreased progress of 3 to [StateProbe match=0 next=1 sentCommit=0 matchCommit=0] - DEBUG 1 [firstindex: 3, commit: 5] sent snapshot[index: 5, term: 1] to 3 [StateProbe match=0 next=1 sentCommit=0 matchCommit=0] - DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 next=6 sentCommit=5 matchCommit=0 paused pendingSnap=5] + DEBUG 1 [firstindex: 3, commit: 6] sent snapshot[index: 6, term: 1] to 3 [StateProbe match=0 next=1 sentCommit=0 matchCommit=0] + DEBUG 1 paused sending replication messages to 3 [StateSnapshot match=0 next=7 sentCommit=6 matchCommit=0 paused pendingSnap=6] 3->1 MsgFortifyLeaderResp Term:1 Log:0/0 LeadEpoch:1 > 1 handling Ready Ready MustSync=false: Messages: - 1->3 MsgApp Term:1 Log:1/3 Commit:5 Entries:[ + 1->3 MsgApp Term:1 Log:1/3 Commit:6 Entries:[ 1/4 EntryConfChangeV2 v2 v3 - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 ] 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + Snapshot: Index:6 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/3 Commit:5 Entries:[ + 1->3 MsgApp Term:1 Log:1/3 Commit:6 Entries:[ 1/4 EntryConfChangeV2 v2 v3 - 1/5 EntryConfChangeV2 + 1/5 EntryNormal "" + 1/6 EntryConfChangeV2 ] DEBUG 3 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 1->3 MsgSnap Term:1 Log:0/0 - Snapshot: Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false - INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 5, term: 1] + Snapshot: Index:6 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + INFO log [committed=0, applied=0, applying=0, unstable.offset=1, unstable.offsetInProgress=1, len(unstable.Entries)=0] starts to restore snapshot [index: 6, term: 1] INFO 3 switched to configuration voters=(1 2 3) - INFO 3 [commit: 5, lastindex: 5, lastterm: 1] restored snapshot [index: 5, term: 1] - INFO 3 [commit: 5] restored snapshot [index: 5, term: 1] + INFO 3 [commit: 6, lastindex: 6, lastterm: 1] restored snapshot [index: 6, term: 1] + INFO 3 [commit: 6] restored snapshot [index: 6, term: 1] > 3 handling Ready Ready MustSync=true: - HardState Term:1 Commit:5 Lead:1 LeadEpoch:1 - Snapshot Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false + HardState Term:1 Commit:6 Lead:1 LeadEpoch:1 + Snapshot Index:6 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false Messages: 3->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) - 3->1 MsgAppResp Term:1 Log:0/5 Commit:5 + 3->1 MsgAppResp Term:1 Log:0/6 Commit:6 > 1 receiving messages 3->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) DEBUG 1 received MsgAppResp(rejected, hint: (index 0, term 0)) from 3 for index 3 - 3->1 MsgAppResp Term:1 Log:0/5 Commit:5 - DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=5 next=6 sentCommit=5 matchCommit=5 paused pendingSnap=5] + 3->1 MsgAppResp Term:1 Log:0/6 Commit:6 + DEBUG 1 recovered from needing snapshot, resumed sending replication messages to 3 [StateSnapshot match=6 next=7 sentCommit=6 matchCommit=6 paused pendingSnap=6] # Nothing else happens. stabilize @@ -256,30 +294,30 @@ stabilize 1 > 1 handling Ready Ready MustSync=true: Entries: - 1/6 EntryConfChangeV2 r2 r3 l2 l3 + 1/7 EntryConfChangeV2 r2 r3 l2 l3 Messages: - 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3 l2 l3] - 1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3 l2 l3] + 1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryConfChangeV2 r2 r3 l2 l3] + 1->3 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryConfChangeV2 r2 r3 l2 l3] # n2, n3 ack them. stabilize 2 3 ---- > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3 l2 l3] + 1->2 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryConfChangeV2 r2 r3 l2 l3] > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/5 Commit:5 Entries:[1/6 EntryConfChangeV2 r2 r3 l2 l3] + 1->3 MsgApp Term:1 Log:1/6 Commit:6 Entries:[1/7 EntryConfChangeV2 r2 r3 l2 l3] > 2 handling Ready Ready MustSync=true: Entries: - 1/6 EntryConfChangeV2 r2 r3 l2 l3 + 1/7 EntryConfChangeV2 r2 r3 l2 l3 Messages: - 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 + 2->1 MsgAppResp Term:1 Log:0/7 Commit:6 > 3 handling Ready Ready MustSync=true: Entries: - 1/6 EntryConfChangeV2 r2 r3 l2 l3 + 1/7 EntryConfChangeV2 r2 r3 l2 l3 Messages: - 3->1 MsgAppResp Term:1 Log:0/6 Commit:5 + 3->1 MsgAppResp Term:1 Log:0/7 Commit:6 # n1 gets some more proposals. This is part of a regression test: There used to # be a bug in which these proposals would prompt the leader to transition out of @@ -299,76 +337,76 @@ stabilize 1 > 1 handling Ready Ready MustSync=true: Entries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" + 1/8 EntryNormal "foo" + 1/9 EntryNormal "bar" Messages: - 1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"] - 1->3 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"] - 1->2 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] - 1->3 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] + 1->2 MsgApp Term:1 Log:1/7 Commit:6 Entries:[1/8 EntryNormal "foo"] + 1->3 MsgApp Term:1 Log:1/7 Commit:6 Entries:[1/8 EntryNormal "foo"] + 1->2 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryNormal "bar"] + 1->3 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryNormal "bar"] > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/6 Commit:5 - 3->1 MsgAppResp Term:1 Log:0/6 Commit:5 + 2->1 MsgAppResp Term:1 Log:0/7 Commit:6 + 3->1 MsgAppResp Term:1 Log:0/7 Commit:6 > 1 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:6 Lead:1 LeadEpoch:1 + HardState Term:1 Vote:1 Commit:7 Lead:1 LeadEpoch:1 CommittedEntries: - 1/6 EntryConfChangeV2 r2 r3 l2 l3 + 1/7 EntryConfChangeV2 r2 r3 l2 l3 Messages: - 1->2 MsgApp Term:1 Log:1/8 Commit:6 - 1->3 MsgApp Term:1 Log:1/8 Commit:6 + 1->2 MsgApp Term:1 Log:1/9 Commit:7 + 1->3 MsgApp Term:1 Log:1/9 Commit:7 INFO 1 switched to configuration voters=(1)&&(1 2 3) learners_next=(2 3) autoleave INFO initiating automatic transition out of joint configuration voters=(1)&&(1 2 3) learners_next=(2 3) autoleave > 1 handling Ready Ready MustSync=true: Entries: - 1/9 EntryConfChangeV2 + 1/10 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryConfChangeV2] - 1->3 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/9 Commit:7 Entries:[1/10 EntryConfChangeV2] + 1->3 MsgApp Term:1 Log:1/9 Commit:7 Entries:[1/10 EntryConfChangeV2] # n2 and n3 also switch to the joint config, and ack the transition out of it. stabilize 2 3 ---- > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"] - 1->2 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] - 1->2 MsgApp Term:1 Log:1/8 Commit:6 - 1->2 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryConfChangeV2] + 1->2 MsgApp Term:1 Log:1/7 Commit:6 Entries:[1/8 EntryNormal "foo"] + 1->2 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryNormal "bar"] + 1->2 MsgApp Term:1 Log:1/9 Commit:7 + 1->2 MsgApp Term:1 Log:1/9 Commit:7 Entries:[1/10 EntryConfChangeV2] > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/6 Commit:5 Entries:[1/7 EntryNormal "foo"] - 1->3 MsgApp Term:1 Log:1/7 Commit:5 Entries:[1/8 EntryNormal "bar"] - 1->3 MsgApp Term:1 Log:1/8 Commit:6 - 1->3 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryConfChangeV2] + 1->3 MsgApp Term:1 Log:1/7 Commit:6 Entries:[1/8 EntryNormal "foo"] + 1->3 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryNormal "bar"] + 1->3 MsgApp Term:1 Log:1/9 Commit:7 + 1->3 MsgApp Term:1 Log:1/9 Commit:7 Entries:[1/10 EntryConfChangeV2] > 2 handling Ready Ready MustSync=true: - HardState Term:1 Commit:6 Lead:1 LeadEpoch:1 + HardState Term:1 Commit:7 Lead:1 LeadEpoch:1 Entries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/8 EntryNormal "foo" + 1/9 EntryNormal "bar" + 1/10 EntryConfChangeV2 CommittedEntries: - 1/6 EntryConfChangeV2 r2 r3 l2 l3 + 1/7 EntryConfChangeV2 r2 r3 l2 l3 Messages: - 2->1 MsgAppResp Term:1 Log:0/7 Commit:5 - 2->1 MsgAppResp Term:1 Log:0/8 Commit:5 2->1 MsgAppResp Term:1 Log:0/8 Commit:6 2->1 MsgAppResp Term:1 Log:0/9 Commit:6 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:7 + 2->1 MsgAppResp Term:1 Log:0/10 Commit:7 INFO 2 switched to configuration voters=(1)&&(1 2 3) learners_next=(2 3) autoleave > 3 handling Ready Ready MustSync=true: - HardState Term:1 Commit:6 Lead:1 LeadEpoch:1 + HardState Term:1 Commit:7 Lead:1 LeadEpoch:1 Entries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/8 EntryNormal "foo" + 1/9 EntryNormal "bar" + 1/10 EntryConfChangeV2 CommittedEntries: - 1/6 EntryConfChangeV2 r2 r3 l2 l3 + 1/7 EntryConfChangeV2 r2 r3 l2 l3 Messages: - 3->1 MsgAppResp Term:1 Log:0/7 Commit:5 - 3->1 MsgAppResp Term:1 Log:0/8 Commit:5 3->1 MsgAppResp Term:1 Log:0/8 Commit:6 3->1 MsgAppResp Term:1 Log:0/9 Commit:6 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:7 + 3->1 MsgAppResp Term:1 Log:0/10 Commit:7 INFO 3 switched to configuration voters=(1)&&(1 2 3) learners_next=(2 3) autoleave # n2 and n3 also leave the joint config and the dust settles. We see at the very @@ -377,65 +415,65 @@ stabilize 2 3 stabilize ---- > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/7 Commit:5 - 2->1 MsgAppResp Term:1 Log:0/8 Commit:5 2->1 MsgAppResp Term:1 Log:0/8 Commit:6 2->1 MsgAppResp Term:1 Log:0/9 Commit:6 - 3->1 MsgAppResp Term:1 Log:0/7 Commit:5 - 3->1 MsgAppResp Term:1 Log:0/8 Commit:5 + 2->1 MsgAppResp Term:1 Log:0/9 Commit:7 + 2->1 MsgAppResp Term:1 Log:0/10 Commit:7 3->1 MsgAppResp Term:1 Log:0/8 Commit:6 3->1 MsgAppResp Term:1 Log:0/9 Commit:6 + 3->1 MsgAppResp Term:1 Log:0/9 Commit:7 + 3->1 MsgAppResp Term:1 Log:0/10 Commit:7 > 1 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:9 Lead:1 LeadEpoch:1 + HardState Term:1 Vote:1 Commit:10 Lead:1 LeadEpoch:1 CommittedEntries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/8 EntryNormal "foo" + 1/9 EntryNormal "bar" + 1/10 EntryConfChangeV2 Messages: - 1->2 MsgApp Term:1 Log:1/9 Commit:7 - 1->3 MsgApp Term:1 Log:1/9 Commit:7 - 1->2 MsgApp Term:1 Log:1/9 Commit:8 - 1->3 MsgApp Term:1 Log:1/9 Commit:8 - 1->2 MsgApp Term:1 Log:1/9 Commit:9 - 1->3 MsgApp Term:1 Log:1/9 Commit:9 + 1->2 MsgApp Term:1 Log:1/10 Commit:8 + 1->3 MsgApp Term:1 Log:1/10 Commit:8 + 1->2 MsgApp Term:1 Log:1/10 Commit:9 + 1->3 MsgApp Term:1 Log:1/10 Commit:9 + 1->2 MsgApp Term:1 Log:1/10 Commit:10 + 1->3 MsgApp Term:1 Log:1/10 Commit:10 INFO 1 switched to configuration voters=(1) learners=(2 3) > 2 receiving messages - 1->2 MsgApp Term:1 Log:1/9 Commit:7 - 1->2 MsgApp Term:1 Log:1/9 Commit:8 - 1->2 MsgApp Term:1 Log:1/9 Commit:9 + 1->2 MsgApp Term:1 Log:1/10 Commit:8 + 1->2 MsgApp Term:1 Log:1/10 Commit:9 + 1->2 MsgApp Term:1 Log:1/10 Commit:10 > 3 receiving messages - 1->3 MsgApp Term:1 Log:1/9 Commit:7 - 1->3 MsgApp Term:1 Log:1/9 Commit:8 - 1->3 MsgApp Term:1 Log:1/9 Commit:9 + 1->3 MsgApp Term:1 Log:1/10 Commit:8 + 1->3 MsgApp Term:1 Log:1/10 Commit:9 + 1->3 MsgApp Term:1 Log:1/10 Commit:10 > 2 handling Ready Ready MustSync=true: - HardState Term:1 Commit:9 Lead:1 LeadEpoch:1 + HardState Term:1 Commit:10 Lead:1 LeadEpoch:1 CommittedEntries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/8 EntryNormal "foo" + 1/9 EntryNormal "bar" + 1/10 EntryConfChangeV2 Messages: - 2->1 MsgAppResp Term:1 Log:0/9 Commit:7 - 2->1 MsgAppResp Term:1 Log:0/9 Commit:8 - 2->1 MsgAppResp Term:1 Log:0/9 Commit:9 + 2->1 MsgAppResp Term:1 Log:0/10 Commit:8 + 2->1 MsgAppResp Term:1 Log:0/10 Commit:9 + 2->1 MsgAppResp Term:1 Log:0/10 Commit:10 INFO 2 switched to configuration voters=(1) learners=(2 3) > 3 handling Ready Ready MustSync=true: - HardState Term:1 Commit:9 Lead:1 LeadEpoch:1 + HardState Term:1 Commit:10 Lead:1 LeadEpoch:1 CommittedEntries: - 1/7 EntryNormal "foo" - 1/8 EntryNormal "bar" - 1/9 EntryConfChangeV2 + 1/8 EntryNormal "foo" + 1/9 EntryNormal "bar" + 1/10 EntryConfChangeV2 Messages: - 3->1 MsgAppResp Term:1 Log:0/9 Commit:7 - 3->1 MsgAppResp Term:1 Log:0/9 Commit:8 - 3->1 MsgAppResp Term:1 Log:0/9 Commit:9 + 3->1 MsgAppResp Term:1 Log:0/10 Commit:8 + 3->1 MsgAppResp Term:1 Log:0/10 Commit:9 + 3->1 MsgAppResp Term:1 Log:0/10 Commit:10 INFO 3 switched to configuration voters=(1) learners=(2 3) > 1 receiving messages - 2->1 MsgAppResp Term:1 Log:0/9 Commit:7 - 2->1 MsgAppResp Term:1 Log:0/9 Commit:8 - 2->1 MsgAppResp Term:1 Log:0/9 Commit:9 - 3->1 MsgAppResp Term:1 Log:0/9 Commit:7 - 3->1 MsgAppResp Term:1 Log:0/9 Commit:8 - 3->1 MsgAppResp Term:1 Log:0/9 Commit:9 + 2->1 MsgAppResp Term:1 Log:0/10 Commit:8 + 2->1 MsgAppResp Term:1 Log:0/10 Commit:9 + 2->1 MsgAppResp Term:1 Log:0/10 Commit:10 + 3->1 MsgAppResp Term:1 Log:0/10 Commit:8 + 3->1 MsgAppResp Term:1 Log:0/10 Commit:9 + 3->1 MsgAppResp Term:1 Log:0/10 Commit:10 diff --git a/pkg/raft/tracker/BUILD.bazel b/pkg/raft/tracker/BUILD.bazel index 4608fc6f2633..51b9edc9b4cb 100644 --- a/pkg/raft/tracker/BUILD.bazel +++ b/pkg/raft/tracker/BUILD.bazel @@ -19,7 +19,6 @@ go_library( "//pkg/raft/raftstoreliveness", "//pkg/util/container/ring", "//pkg/util/hlc", - "//pkg/util/syncutil", "@com_github_cockroachdb_redact//:redact", "@org_golang_x_exp//maps", ], diff --git a/pkg/raft/tracker/fortificationtracker.go b/pkg/raft/tracker/fortificationtracker.go index 8e66c3eabe48..aaae752d2c10 100644 --- a/pkg/raft/tracker/fortificationtracker.go +++ b/pkg/raft/tracker/fortificationtracker.go @@ -15,7 +15,6 @@ import ( pb "github.com/cockroachdb/cockroach/pkg/raft/raftpb" "github.com/cockroachdb/cockroach/pkg/raft/raftstoreliveness" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) // FortificationTracker is used to track fortification from peers. This can @@ -60,11 +59,6 @@ type FortificationTracker struct { // never regresses for a raft group. Naively, without any tracking, this // can happen around configuration changes[1] and leader step down[2]. // - // NB: We use an atomicTimestamp here, which allows us to forward - // leadMaxSupported on every call to LeadSupportUntil, without requiring - // callers to acquire a write lock. Typically, LeadSupportUntil is called into - // by get{LeadSupport,}Status - // // [1] We must ensure that the current LeadSupportUntil is greater than or // equal to any previously calculated LeadSupportUntil before proposing a new // configuration change. @@ -73,7 +67,7 @@ type FortificationTracker struct { // de-fortification messages, voting for another peer, or calling an election // at a higher term) that could elect a leader until LeadSupportUntil is in // the past. - leaderMaxSupported atomicTimestamp + leaderMaxSupported hlc.Timestamp logger raftlogger.Logger @@ -94,11 +88,14 @@ type FortificationTracker struct { steppingDownTerm uint64 // computedLeadSupportUntil is the last computed LeadSupportUntil. We - // update this value on (1) Every tick, (2) Every time a new fortification is - // recorded, and (3) On config changes. Callers of LeadSupportUntil will get - // this cached version of LeadSupportUntil, which is good because - // LeadSupportUntil is called by every request trying to evaluate the lease's - // status. + // update this value on: + // 1. Every tick. + // 2. Every time a new fortification is recorded. + // 3. On config changes. + // + // Callers of LeadSupportUntil will get this cached version of + // LeadSupportUntil, which is useful because LeadSupportUntil is called by + // every request trying to evaluate the lease's status. computedLeadSupportUntil hlc.Timestamp // supportExpMap is a map that hangs off the fortificationTracker to prevent @@ -193,29 +190,26 @@ func (ft *FortificationTracker) IsFortifiedBy(id pb.PeerID) (isFortified bool, i // LeadSupportUntil returns the timestamp until which the leader is guaranteed // fortification until based on the fortification being tracked for it by its // peers. -func (ft *FortificationTracker) LeadSupportUntil(state pb.StateType) hlc.Timestamp { - // TODO(ibrahim): Consider removing the state from the function parameters. - if state != pb.StateLeader || ft.steppingDown { - // If we're not the leader or if we are intending to step down, we shouldn't - // advance LeadSupportUntil. - return ft.leaderMaxSupported.Load() - } - - // Forward the leaderMaxSupported to avoid regressions when the configuration - // changes. - return ft.leaderMaxSupported.Forward(ft.computedLeadSupportUntil) +func (ft *FortificationTracker) LeadSupportUntil() hlc.Timestamp { + return ft.leaderMaxSupported } // ComputeLeadSupportUntil updates the field // computedLeadSupportUntil by computing the LeadSupportExpiration. -func (ft *FortificationTracker) ComputeLeadSupportUntil(state pb.StateType) hlc.Timestamp { +func (ft *FortificationTracker) ComputeLeadSupportUntil(state pb.StateType) { if state != pb.StateLeader { panic("ComputeLeadSupportUntil should only be called by the leader") } if len(ft.fortification) == 0 { ft.computedLeadSupportUntil = hlc.Timestamp{} - return ft.computedLeadSupportUntil // fast-path for no fortification + return // fast-path for no fortification + } + + if ft.steppingDown { + // We're in the process of stepping down, so we don't want to advance the + // LeadSupportUntil; early return. + return } clear(ft.supportExpMap) @@ -233,7 +227,10 @@ func (ft *FortificationTracker) ComputeLeadSupportUntil(state pb.StateType) hlc. } }) ft.computedLeadSupportUntil = ft.config.Voters.LeadSupportExpiration(ft.supportExpMap) - return ft.computedLeadSupportUntil + + // Forward the leaderMaxSupported to avoid regressions when the configuration + // changes. + ft.leaderMaxSupported.Forward(ft.computedLeadSupportUntil) } // CanDefortify returns whether the caller can safely[1] de-fortify the term @@ -246,14 +243,13 @@ func (ft *FortificationTracker) CanDefortify() bool { if ft.term == 0 { return false // nothing is being tracked } - leaderMaxSupported := ft.leaderMaxSupported.Load() - if leaderMaxSupported.IsEmpty() { + if ft.leaderMaxSupported.IsEmpty() { // If leaderMaxSupported is empty, it means that we've never returned any // timestamps to the layers above in calls to LeadSupportUntil. We should be // able to de-fortify. If a tree falls in a forrest ... ft.logger.Debugf("leaderMaxSupported is empty when computing whether we can de-fortify or not") } - return ft.storeLiveness.SupportExpired(leaderMaxSupported) + return ft.storeLiveness.SupportExpired(ft.leaderMaxSupported) } // NeedsDefortify returns whether the node should still continue to broadcast @@ -370,14 +366,14 @@ func (ft *FortificationTracker) ConfigChangeSafe() bool { // previous configuration, which is reflected in leaderMaxSupported. // // NB: Only run by the leader. - return ft.leaderMaxSupported.Load().LessEq(ft.computedLeadSupportUntil) + return ft.leaderMaxSupported.LessEq(ft.computedLeadSupportUntil) } // QuorumActive returns whether the leader is currently supported by a quorum or // not. func (ft *FortificationTracker) QuorumActive() bool { // NB: Only run by the leader. - return !ft.storeLiveness.SupportExpired(ft.LeadSupportUntil(pb.StateLeader)) + return !ft.storeLiveness.SupportExpired(ft.LeadSupportUntil()) } // RequireQuorumSupportOnCampaign returns true if quorum support before @@ -425,30 +421,3 @@ func (ft *FortificationTracker) String() string { } return buf.String() } - -// atomicTimestamp is a thin wrapper to provide atomic access to a timestamp. -type atomicTimestamp struct { - mu syncutil.Mutex - - ts hlc.Timestamp -} - -func (a *atomicTimestamp) Load() hlc.Timestamp { - a.mu.Lock() - defer a.mu.Unlock() - return a.ts -} - -func (a *atomicTimestamp) Forward(ts hlc.Timestamp) hlc.Timestamp { - a.mu.Lock() - defer a.mu.Unlock() - - a.ts.Forward(ts) - return a.ts -} - -func (a *atomicTimestamp) Reset() { - a.mu.Lock() - defer a.mu.Unlock() - a.ts = hlc.Timestamp{} -} diff --git a/pkg/raft/tracker/fortificationtracker_test.go b/pkg/raft/tracker/fortificationtracker_test.go index 1ee544d663bf..4d583badc2e5 100644 --- a/pkg/raft/tracker/fortificationtracker_test.go +++ b/pkg/raft/tracker/fortificationtracker_test.go @@ -169,34 +169,6 @@ func TestLeadSupportUntil(t *testing.T) { initLeadSupportedUntil: hlc.Timestamp{}, expTS: hlc.Timestamp{}, }, - { - ids: []pb.PeerID{1, 2, 3}, - storeLiveness: mockLiveness3Peers, - setup: func(fortificationTracker *FortificationTracker) { - fortificationTracker.RecordFortification(1, 10) - fortificationTracker.RecordFortification(3, 30) - fortificationTracker.RecordFortification(2, 20) - }, - // If the state isn't StateLeader, we expect the LeadSupportUntil won't - // be computed, and the previous value will be returned. - state: pb.StateFollower, - initLeadSupportedUntil: ts(1), - expTS: ts(1), - }, - { - ids: []pb.PeerID{1, 2, 3}, - storeLiveness: mockLiveness3Peers, - setup: func(fortificationTracker *FortificationTracker) { - fortificationTracker.RecordFortification(1, 10) - fortificationTracker.RecordFortification(3, 30) - fortificationTracker.RecordFortification(2, 20) - }, - // If the state isn't StateLeader, we expect that LeadSupportUntil won't - // be computed, and the previous value will be returned. - state: pb.StateCandidate, - initLeadSupportedUntil: ts(1), - expTS: ts(1), - }, { ids: []pb.PeerID{1, 2, 3}, storeLiveness: mockLiveness3Peers, @@ -223,7 +195,8 @@ func TestLeadSupportUntil(t *testing.T) { tc.setup(fortificationTracker) fortificationTracker.leaderMaxSupported.Forward(tc.initLeadSupportedUntil) - require.Equal(t, tc.expTS, fortificationTracker.LeadSupportUntil(tc.state)) + fortificationTracker.ComputeLeadSupportUntil(tc.state) + require.Equal(t, tc.expTS, fortificationTracker.LeadSupportUntil()) } } @@ -514,7 +487,7 @@ func TestQuorumActive(t *testing.T) { tc.setup(fortificationTracker) require.Equal(t, tc.expQuorumActive, fortificationTracker.QuorumActive(), "#%d %s %s", - i, fortificationTracker.LeadSupportUntil(pb.StateLeader), tc.curTS) + i, fortificationTracker.LeadSupportUntil(), tc.curTS) } } @@ -745,9 +718,9 @@ func TestCanDefortify(t *testing.T) { ft.RecordFortification(3, 30) }, curTS: ts(10), - // LeadSupportUntil = ts(15); however, because we don't call it explicitly, - // we should be able to de-fortify. - expCanDefortify: true, + // LeadSupportUntil = ts(15); even if we don't call it explicitly, + // we should not be able to de-fortify. + expCanDefortify: false, }, { setup: func(ft *FortificationTracker) { @@ -769,7 +742,7 @@ func TestCanDefortify(t *testing.T) { ft.Reset(10) // set non-zero term tc.setup(ft) if !tc.expLeadSupportUntil.IsEmpty() { - require.Equal(t, tc.expLeadSupportUntil, ft.LeadSupportUntil(pb.StateLeader)) + require.Equal(t, tc.expLeadSupportUntil, ft.LeadSupportUntil()) } require.Equal(t, tc.expCanDefortify, ft.CanDefortify()) } @@ -893,7 +866,7 @@ func TestConfigChangeSafe(t *testing.T) { ft.RecordFortification(1, 10) ft.RecordFortification(2, 20) ft.RecordFortification(3, 30) - require.Equal(t, ts(15), ft.LeadSupportUntil(pb.StateLeader)) + require.Equal(t, ts(15), ft.LeadSupportUntil()) // Perform a configuration change that adds r4 to the voter set. cfg.Voters[0][4] = struct{}{} @@ -902,7 +875,7 @@ func TestConfigChangeSafe(t *testing.T) { ft.ComputeLeadSupportUntil(pb.StateLeader) require.Equal(t, tc.expConfigChangeSafe, ft.ConfigChangeSafe()) - require.Equal(t, tc.expLeadSupportUntil, ft.LeadSupportUntil(pb.StateLeader)) + require.Equal(t, tc.expLeadSupportUntil, ft.LeadSupportUntil()) } } From 77b879cec42576cc5b6913ed51efc21b40ee70ba Mon Sep 17 00:00:00 2001 From: Jeff Swenson Date: Mon, 6 Jan 2025 10:32:16 -0500 Subject: [PATCH 103/126] crosscluster: add `crdb_route` parameter for LDR and PCR The `crdb_route` query parameter determines how the destination cluster's stream processor connects to the source cluster. There are two options for the query parameter: "node" and "gateway". Here is an example of using the route paraemeter to create an external connection that is usable for LDR or PCR. ```SQL -- A connection that routes all replication traffic via the configured -- connection URI. CREATE EXTERNAL CONNECTION 'external://source-db' AS 'postgresql://user:password@somedb.com:26257/sslmode=verify-full&crdb_route=gateway' -- A connection that enumerates nodes in the source cluster and connects -- directly to nodes. CREATE EXTERNAL CONNECTION 'external://source-db' AS 'postgresql://user:password@somedb.com:26257/sslmode=verify-full&crdb_route=node' ``` The "node" option is the original and default behavior. The "node" option requires the source and destination clusters to be in the same IP network. The way it works is the connection string supplied to LDR and PCR is used to connect to the source cluster and generate a physical sql plan for the replication. The physical plan includes the `--sql-addvertise-addr` for nodes in the source cluster and processors in the destination cluster connect directly to the nodes. Using the "node" routing is ideal because there are no extra network hops and the source cluster can control how load is distributed across its nodes. The "gateway" option is a new option that is introduced in order to support routing PCR and LDR over a load balancer. When specified, the destination cluster ignores the node addresses returned by the physical plan and instead opens a connection for each processor to the URI supplied by the user. This introduces an extra network hop and does not distribute load as evenly, but it works in deployments where the source cluster is only reachable over a load balancer. Routing over a load balancer only requires changing the destination clusters behavior. Nodes in the source cluster were always implemented to act as a gateway and serve rangefeeds that are backed by data stored on different nodes. This support exists so that the cross cluster replication does not need to re-plan every time a range moves to a different node. Release note (sql change): LDR and PCR may use the `crdb_route=gateway` query option to route the replication streams over a load balancer. Epic: CRDB-40896 --- pkg/crosscluster/logical/BUILD.bazel | 1 + .../logical/dead_letter_queue_test.go | 2 + .../logical/logical_replication_job.go | 19 +- .../logical/logical_replication_job_test.go | 208 ++++++++---------- .../logical/udf_row_processor_test.go | 9 +- pkg/crosscluster/physical/BUILD.bazel | 1 + .../physical/alter_replication_job_test.go | 4 +- .../physical/replication_stream_e2e_test.go | 3 +- .../physical/stream_ingestion_dist.go | 31 +-- .../physical/stream_ingestion_job_test.go | 86 ++++++-- .../stream_ingestion_planning_test.go | 5 +- .../producer/replication_manager.go | 13 +- pkg/crosscluster/producer/stream_lifetime.go | 16 +- .../replicationtestutils/BUILD.bazel | 2 + .../replicationtestutils/testutils.go | 51 +++-- .../replicationtestutils/uri_util.go | 50 +++++ .../streamclient/partitioned_stream_client.go | 15 +- pkg/crosscluster/streamclient/uri.go | 52 ++++- pkg/crosscluster/streamclient/uri_test.go | 3 + pkg/sql/exec_util.go | 2 + 20 files changed, 357 insertions(+), 216 deletions(-) create mode 100644 pkg/crosscluster/replicationtestutils/uri_util.go diff --git a/pkg/crosscluster/logical/BUILD.bazel b/pkg/crosscluster/logical/BUILD.bazel index e84738aa1213..60304f80292b 100644 --- a/pkg/crosscluster/logical/BUILD.bazel +++ b/pkg/crosscluster/logical/BUILD.bazel @@ -162,6 +162,7 @@ go_test( "//pkg/testutils/skip", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", + "//pkg/util", "//pkg/util/allstacks", "//pkg/util/hlc", "//pkg/util/leaktest", diff --git a/pkg/crosscluster/logical/dead_letter_queue_test.go b/pkg/crosscluster/logical/dead_letter_queue_test.go index b341bb30ab45..ac7192731408 100644 --- a/pkg/crosscluster/logical/dead_letter_queue_test.go +++ b/pkg/crosscluster/logical/dead_letter_queue_test.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" + "github.com/cockroachdb/cockroach/pkg/crosscluster/replicationtestutils" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" @@ -505,6 +506,7 @@ func testEndToEndDLQ(t *testing.T, mode string) { dbA.Exec(t, "SET CLUSTER SETTING logical_replication.consumer.retry_queue_duration = '100ms'") dbA.Exec(t, "SET CLUSTER SETTING logical_replication.consumer.retry_queue_backoff = '1ms'") + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) type testCase struct { tableName string diff --git a/pkg/crosscluster/logical/logical_replication_job.go b/pkg/crosscluster/logical/logical_replication_job.go index 467c8d4b985d..b72eff050b4a 100644 --- a/pkg/crosscluster/logical/logical_replication_job.go +++ b/pkg/crosscluster/logical/logical_replication_job.go @@ -190,13 +190,18 @@ func (r *logicalReplicationResumer) ingest( if err != nil { return err } - if err := r.job.NoTxn().Update(ctx, func(txn isql.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { - ldrProg := md.Progress.Details.(*jobspb.Progress_LogicalReplication).LogicalReplication - ldrProg.PartitionConnUris = planInfo.partitionPgUrls - ju.UpdateProgress(md.Progress) - return nil - }); err != nil { - return err + + // If the routing mode is gateway, we don't want to checkpoint addresses + // since they may not be in the same network. + if uris[0].RoutingMode() != streamclient.RoutingModeGateway { + if err := r.job.NoTxn().Update(ctx, func(txn isql.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + ldrProg := md.Progress.Details.(*jobspb.Progress_LogicalReplication).LogicalReplication + ldrProg.PartitionConnUris = planInfo.partitionPgUrls + ju.UpdateProgress(md.Progress) + return nil + }); err != nil { + return err + } } // Update the local progress copy as it was just updated. progress = r.job.Progress().Details.(*jobspb.Progress_LogicalReplication).LogicalReplication diff --git a/pkg/crosscluster/logical/logical_replication_job_test.go b/pkg/crosscluster/logical/logical_replication_job_test.go index 6de3ed461f2b..0617f8b1e67f 100644 --- a/pkg/crosscluster/logical/logical_replication_job_test.go +++ b/pkg/crosscluster/logical/logical_replication_job_test.go @@ -10,6 +10,7 @@ import ( "context" gosql "database/sql" "fmt" + "net" "net/url" "slices" "strings" @@ -49,6 +50,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/allstacks" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -159,8 +161,8 @@ func TestLogicalStreamIngestionJobNameResolution(t *testing.T) { server, s, dbA, dbB := setupLogicalTestServer(t, ctx, testClusterBaseClusterArgs, 1) defer server.Stopper().Stop(ctx) - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) for _, c := range cases { t.Run(c.name, func(t *testing.T) { @@ -205,7 +207,6 @@ func TestLogicalStreamIngestionJob(t *testing.T) { t.Run(mode, func(t *testing.T) { testLogicalStreamIngestionJobBasic(t, mode) }) - } } @@ -270,18 +271,8 @@ func testLogicalStreamIngestionJobBasic(t *testing.T, mode string) { dbA.Exec(t, "INSERT INTO tab VALUES (1, 'hello')") dbB.Exec(t, "INSERT INTO tab VALUES (1, 'goodbye')") - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() - - // Swap one of the URLs to external:// to verify this indirection works. - // TODO(dt): this create should support placeholder for URI. - dbB.Exec(t, "CREATE EXTERNAL CONNECTION a AS '"+dbAURL.String()+"'") - dbAURL = url.URL{ - Scheme: "external", - Host: "a", - } + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) var ( jobAID jobspb.JobID @@ -324,18 +315,8 @@ func TestLogicalStreamIngestionJobWithCursor(t *testing.T) { dbA.Exec(t, "INSERT INTO tab VALUES (1, 'hello')") dbB.Exec(t, "INSERT INTO tab VALUES (1, 'goodbye')") - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() - - // Swap one of the URLs to external:// to verify this indirection works. - // TODO(dt): this create should support placeholder for URI. - dbB.Exec(t, "CREATE EXTERNAL CONNECTION a AS '"+dbAURL.String()+"'") - dbAURL = url.URL{ - Scheme: "external", - Host: "a", - } + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) var ( jobAID jobspb.JobID @@ -390,8 +371,7 @@ func TestCreateTables(t *testing.T) { defer tc.Stopper().Stop(ctx) sqlA := sqlDBs[0] - aURL, cleanup := srv.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + aURL := replicationtestutils.GetReplicationUri(t, srv, srv, serverutils.DBName("a")) t.Run("basic", func(t *testing.T) { // Ensure the offline scan replicates index spans. @@ -519,18 +499,8 @@ func TestLogicalStreamIngestionAdvancePTS(t *testing.T) { dbA.Exec(t, "INSERT INTO tab VALUES (1, 'hello')") dbB.Exec(t, "INSERT INTO tab VALUES (1, 'goodbye')") - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() - - // Swap one of the URLs to external:// to verify this indirection works. - // TODO(dt): this create should support placeholder for URI. - dbB.Exec(t, "CREATE EXTERNAL CONNECTION a AS '"+dbAURL.String()+"'") - dbAURL = url.URL{ - Scheme: "external", - Host: "a", - } + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) var ( jobAID jobspb.JobID @@ -565,8 +535,7 @@ func TestLogicalStreamIngestionCancelUpdatesProducerJob(t *testing.T) { dbA.Exec(t, "SET CLUSTER SETTING physical_replication.producer.stream_liveness_track_frequency='50ms'") - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) var jobBID jobspb.JobID dbB.QueryRow(t, "CREATE LOGICAL REPLICATION STREAM FROM TABLE tab ON $1 INTO TABLE tab", dbAURL.String()).Scan(&jobBID) @@ -595,8 +564,7 @@ func TestRestoreFromLDR(t *testing.T) { server, s, dbA, dbB := setupLogicalTestServer(t, ctx, args, 1) defer server.Stopper().Stop(ctx) - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) var jobBID jobspb.JobID dbA.Exec(t, "INSERT INTO tab VALUES (1, 'hello')") @@ -626,8 +594,7 @@ func TestImportIntoLDR(t *testing.T) { server, s, dbA, dbB := setupLogicalTestServer(t, ctx, args, 1) defer server.Stopper().Stop(ctx) - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) var jobBID jobspb.JobID dbA.Exec(t, "INSERT INTO tab VALUES (1, 'hello')") @@ -651,8 +618,7 @@ func TestLogicalStreamIngestionErrors(t *testing.T) { server := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) defer server.Stopper().Stop(ctx) s := server.Server(0).ApplicationLayer() - url, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + url := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) urlA := url.String() _, err := server.Conns[0].Exec("CREATE DATABASE a") @@ -715,9 +681,8 @@ family f2(other_payload, v2)) serverASQL.Exec(t, "INSERT INTO tab_with_cf(pk, payload, other_payload) VALUES (1, 'hello', 'ruroh1')") - serverAURL, cleanup := s.PGUrl(t) + serverAURL := replicationtestutils.GetReplicationUri(t, s, s) serverAURL.Path = "a" - defer cleanup() var jobBID jobspb.JobID serverBSQL.QueryRow(t, "CREATE LOGICAL REPLICATION STREAM FROM TABLE tab_with_cf ON $1 INTO TABLE tab_with_cf WITH MODE = validated", serverAURL.String()).Scan(&jobBID) @@ -748,9 +713,7 @@ func TestLogicalReplicationWithPhantomDelete(t *testing.T) { tc, s, serverASQL, serverBSQL := setupLogicalTestServer(t, ctx, testClusterBaseClusterArgs, 1) defer tc.Stopper().Stop(ctx) - serverAURL, cleanup := s.PGUrl(t) - serverAURL.Path = "a" - defer cleanup() + serverAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) for _, mode := range []string{"validated", "immediate"} { t.Run(mode, func(t *testing.T) { @@ -789,10 +752,8 @@ func TestFilterRangefeedInReplicationStream(t *testing.T) { dbA, dbB, dbC := dbs[0], dbs[1], dbs[2] - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) var jobAID, jobBID, jobCID jobspb.JobID @@ -857,8 +818,7 @@ func TestRandomTables(t *testing.T) { var tableName, streamStartStmt string rng, _ := randutil.NewPseudoRand() - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) // Keep retrying until the random table satisfies all the static checks // we make when creating the replication stream. @@ -1013,8 +973,7 @@ func TestPreviouslyInterestingTables(t *testing.T) { baseTableName := "rand_table" rng, _ := randutil.NewPseudoRand() numInserts := 20 - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) for i, tc := range testCases { t.Run(tc.name, func(t *testing.T) { tableName := fmt.Sprintf("%s%d", baseTableName, i) @@ -1104,10 +1063,8 @@ func TestLogicalAutoReplan(t *testing.T) { serverutils.SetClusterSetting(t, server, "logical_replication.replan_flow_threshold", 0) serverutils.SetClusterSetting(t, server, "logical_replication.replan_flow_frequency", time.Millisecond*500) - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) var ( jobAID jobspb.JobID @@ -1170,8 +1127,7 @@ func TestLogicalJobResiliency(t *testing.T) { server, s, dbA, dbB := setupLogicalTestServer(t, ctx, clusterArgs, 3) defer server.Stopper().Stop(ctx) - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) CreateScatteredTable(t, dbB, 2, "B") @@ -1222,10 +1178,8 @@ func TestHeartbeatCancel(t *testing.T) { serverutils.SetClusterSetting(t, server, "logical_replication.consumer.heartbeat_frequency", time.Second*1) - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) var ( jobAID jobspb.JobID @@ -1270,8 +1224,7 @@ func TestMultipleSourcesIntoSingleDest(t *testing.T) { server, s, runners, dbNames := setupServerWithNumDBs(t, ctx, clusterArgs, 1, 3) defer server.Stopper().Stop(ctx) - PGURLs, cleanup := GetPGURLs(t, s, dbNames) - defer cleanup() + PGURLs := GetPGURLs(t, s, dbNames) dbA, dbB, dbC := runners[0], runners[1], runners[2] @@ -1358,8 +1311,7 @@ func TestFourWayReplication(t *testing.T) { server, s, runners, dbNames := setupServerWithNumDBs(t, ctx, clusterArgs, 1, numDBs) defer server.Stopper().Stop(ctx) - PGURLs, cleanup := GetPGURLs(t, s, dbNames) - defer cleanup() + PGURLs := GetPGURLs(t, s, dbNames) // Each row is a DB, each column is a jobID from another DB to that target DB jobIDs := make([][]jobspb.JobID, numDBs) @@ -1415,8 +1367,7 @@ func TestForeignKeyConstraints(t *testing.T) { server, s, dbA, _ := setupLogicalTestServer(t, ctx, clusterArgs, 1) defer server.Stopper().Stop(ctx) - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) dbA.Exec(t, "CREATE TABLE test(a int primary key, b int)") @@ -1563,22 +1514,13 @@ func CreateScatteredTable(t *testing.T, db *sqlutils.SQLRunner, numNodes int, db }, timeout) } -func GetPGURLs( - t *testing.T, s serverutils.ApplicationLayerInterface, dbNames []string, -) ([]url.URL, func()) { +func GetPGURLs(t *testing.T, s serverutils.ApplicationLayerInterface, dbNames []string) []url.URL { result := []url.URL{} - cleanups := []func(){} for _, name := range dbNames { - resultURL, cleanup := s.PGUrl(t, serverutils.DBName(name)) + resultURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName(name)) result = append(result, resultURL) - cleanups = append(cleanups, cleanup) - } - - return result, func() { - for _, f := range cleanups { - f() - } } + return result } func WaitUntilReplicatedTime( @@ -1728,18 +1670,8 @@ func TestLogicalStreamIngestionJobWithFallbackUDF(t *testing.T) { dbB.Exec(t, lwwFunc) dbA.Exec(t, lwwFunc) - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() - - // Swap one of the URLs to external:// to verify this indirection works. - // TODO(dt): this create should support placeholder for URI. - dbB.Exec(t, "CREATE EXTERNAL CONNECTION a AS '"+dbAURL.String()+"'") - dbAURL = url.URL{ - Scheme: "external", - Host: "a", - } + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) var ( jobAID jobspb.JobID @@ -1857,15 +1789,8 @@ func TestShowLogicalReplicationJobs(t *testing.T) { server, s, dbA, dbB := setupLogicalTestServer(t, ctx, testClusterBaseClusterArgs, 1) defer server.Stopper().Stop(ctx) - dbAURL, cleanup := s.PGUrl(t, - serverutils.DBName("a"), - serverutils.UserPassword(username.RootUser, "password")) - defer cleanup() - - dbBURL, cleanupB := s.PGUrl(t, - serverutils.DBName("b"), - serverutils.UserPassword(username.RootUser, "password")) - defer cleanupB() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a"), serverutils.UserPassword(username.RootUser, "password")) + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b"), serverutils.UserPassword(username.RootUser, "password")) redactedDbAURL := strings.Replace(dbAURL.String(), "password", `redacted`, 1) redactedDbBURL := strings.Replace(dbBURL.String(), "password", `redacted`, 1) @@ -2002,8 +1927,7 @@ func TestUserPrivileges(t *testing.T) { server, s, dbA, _ := setupLogicalTestServer(t, ctx, clusterArgs, 1) defer server.Stopper().Stop(ctx) - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) // Create user with no privileges dbA.Exec(t, fmt.Sprintf("CREATE USER %s", username.TestUser)) @@ -2081,8 +2005,7 @@ func TestLogicalReplicationSchemaChanges(t *testing.T) { server, s, dbA, dbB := setupLogicalTestServer(t, ctx, clusterArgs, 1) defer server.Stopper().Stop(ctx) - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) var jobAID jobspb.JobID dbA.QueryRow(t, "CREATE LOGICAL REPLICATION STREAM FROM TABLE tab ON $1 INTO TABLE tab", dbBURL.String()).Scan(&jobAID) @@ -2123,11 +2046,7 @@ func TestUserDefinedTypes(t *testing.T) { server, s, dbA, dbB := setupLogicalTestServer(t, ctx, clusterArgs, 1) defer server.Stopper().Stop(ctx) - _, cleanupA := s.PGUrl(t, serverutils.DBName("a")) - defer cleanupA() - - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) // Create the same user-defined type both tables. dbA.Exec(t, "CREATE TYPE my_enum AS ENUM ('one', 'two', 'three')") @@ -2172,6 +2091,52 @@ func TestUserDefinedTypes(t *testing.T) { } } +func TestLogicalReplicationGatewayRoute(t *testing.T) { + defer leaktest.AfterTest(t)() + + // Create a blackhole so we can claim a port and black hole any connections + // routed there. + blackhole, err := net.Listen("tcp", "localhost:0") + require.NoError(t, err) + defer func() { + require.NoError(t, blackhole.Close()) + }() + + t.Log("blackhole listening on", blackhole.Addr()) + // Set the SQL advertise addr to something unroutable so that we know the + // config connection url was used for all streams. + args := testClusterBaseClusterArgs + args.ServerArgs.Knobs.Streaming = &sql.StreamingTestingKnobs{ + OnGetSQLInstanceInfo: func(node *roachpb.NodeDescriptor) *roachpb.NodeDescriptor { + copy := *node + copy.SQLAddress = util.UnresolvedAddr{ + NetworkField: "tcp", + AddressField: blackhole.Addr().String(), + } + return © + }, + } + ts, s, runners, dbs := setupServerWithNumDBs(t, context.Background(), args, 1, 2) + defer ts.Stopper().Stop(context.Background()) + + url, cleanup := s.PGUrl(t, serverutils.DBName(dbs[1])) + defer cleanup() + + q := url.Query() + q.Set(streamclient.RoutingModeKey, string(streamclient.RoutingModeGateway)) + url.RawQuery = q.Encode() + + var jobID jobspb.JobID + runners[0].QueryRow(t, "CREATE LOGICAL REPLICATION STREAM FROM TABLE tab ON $1 INTO TABLE tab", url.String()).Scan(&jobID) + runners[1].Exec(t, "INSERT INTO tab VALUES (1, 'hello')") + + now := s.Clock().Now() + WaitUntilReplicatedTime(t, now, runners[0], jobID) + + progress := jobutils.GetJobProgress(t, runners[0], jobID) + require.Empty(t, progress.Details.(*jobspb.Progress_LogicalReplication).LogicalReplication.PartitionConnUris) +} + // TestLogicalReplicationCreationChecks verifies that we check that the table // schemas are compatible when creating the replication stream. func TestLogicalReplicationCreationChecks(t *testing.T) { @@ -2192,8 +2157,7 @@ func TestLogicalReplicationCreationChecks(t *testing.T) { server, s, dbA, dbB := setupLogicalTestServer(t, ctx, clusterArgs, 1) defer server.Stopper().Stop(ctx) - dbBURL, cleanupB := s.PGUrl(t, serverutils.DBName("b")) - defer cleanupB() + dbBURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("b")) // Column families are not allowed. dbA.Exec(t, "ALTER TABLE tab ADD COLUMN new_col INT NOT NULL CREATE FAMILY f1") diff --git a/pkg/crosscluster/logical/udf_row_processor_test.go b/pkg/crosscluster/logical/udf_row_processor_test.go index 391552e69552..214b1497e657 100644 --- a/pkg/crosscluster/logical/udf_row_processor_test.go +++ b/pkg/crosscluster/logical/udf_row_processor_test.go @@ -85,8 +85,7 @@ func TestUDFWithRandomTables(t *testing.T) { sqlA, tableName, numInserts, nil) require.NoError(t, err) - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) streamStartStmt := fmt.Sprintf("CREATE LOGICAL REPLICATION STREAM FROM TABLE %[1]s ON $1 INTO TABLE %[1]s WITH FUNCTION repl_apply FOR TABLE %[1]s", tableName) var jobBID jobspb.JobID @@ -127,8 +126,7 @@ func TestUDFInsertOnly(t *testing.T) { $$ LANGUAGE plpgsql `) - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) streamStartStmt := fmt.Sprintf("CREATE LOGICAL REPLICATION STREAM FROM TABLE %[1]s ON $1 INTO TABLE %[1]s WITH DEFAULT FUNCTION = 'funcs.repl_apply'", tableName) var jobBID jobspb.JobID @@ -177,8 +175,7 @@ func TestUDFPreviousValue(t *testing.T) { $$ LANGUAGE plpgsql `) - dbAURL, cleanup := s.PGUrl(t, serverutils.DBName("a")) - defer cleanup() + dbAURL := replicationtestutils.GetReplicationUri(t, s, s, serverutils.DBName("a")) streamStartStmt := fmt.Sprintf("CREATE LOGICAL REPLICATION STREAM FROM TABLE %[1]s ON $1 INTO TABLE %[1]s WITH FUNCTION repl_apply FOR TABLE %[1]s", tableName) var jobBID jobspb.JobID diff --git a/pkg/crosscluster/physical/BUILD.bazel b/pkg/crosscluster/physical/BUILD.bazel index 35ec16478b67..c54f9119c611 100644 --- a/pkg/crosscluster/physical/BUILD.bazel +++ b/pkg/crosscluster/physical/BUILD.bazel @@ -178,6 +178,7 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/testutils/storageutils", "//pkg/testutils/testcluster", + "//pkg/util", "//pkg/util/ctxgroup", "//pkg/util/duration", "//pkg/util/hlc", diff --git a/pkg/crosscluster/physical/alter_replication_job_test.go b/pkg/crosscluster/physical/alter_replication_job_test.go index c1eb3b1d7045..c93df515c740 100644 --- a/pkg/crosscluster/physical/alter_replication_job_test.go +++ b/pkg/crosscluster/physical/alter_replication_job_test.go @@ -8,7 +8,6 @@ package physical import ( "context" "fmt" - "net/url" "testing" "time" @@ -586,8 +585,7 @@ func TestAlterTenantStartReplicationAfterRestore(t *testing.T) { enforcedGC.ts = afterBackup enforcedGC.Unlock() - u, cleanupURLA := sqlutils.PGUrl(t, srv.SQLAddr(), t.Name(), url.User(username.RootUser)) - defer cleanupURLA() + u := replicationtestutils.GetReplicationUri(t, srv, srv, serverutils.User(username.RootUser)) db.Exec(t, "RESTORE TENANT 3 FROM LATEST IN 'nodelocal://1/t' WITH TENANT = '5', TENANT_NAME = 't2'") db.Exec(t, "ALTER TENANT t2 START REPLICATION OF t1 ON $1", u.String()) diff --git a/pkg/crosscluster/physical/replication_stream_e2e_test.go b/pkg/crosscluster/physical/replication_stream_e2e_test.go index 4594ae64d3c1..2f379c099439 100644 --- a/pkg/crosscluster/physical/replication_stream_e2e_test.go +++ b/pkg/crosscluster/physical/replication_stream_e2e_test.go @@ -679,6 +679,7 @@ func TestTenantStreamingMultipleNodes(t *testing.T) { testutils.RunTrueAndFalse(t, "fromSystem", func(t *testing.T, sys bool) { args := replicationtestutils.DefaultTenantStreamingClustersArgs args.MultitenantSingleClusterNumNodes = 3 + args.RoutingMode = streamclient.RoutingModeNode // Track the number of unique addresses that were connected to clientAddresses := make(map[string]struct{}) @@ -787,6 +788,7 @@ func TestStreamingAutoReplan(t *testing.T) { ctx := context.Background() args := replicationtestutils.DefaultTenantStreamingClustersArgs args.MultitenantSingleClusterNumNodes = 1 + args.RoutingMode = streamclient.RoutingModeNode retryErrorChan := make(chan error) turnOffReplanning := make(chan struct{}) @@ -802,7 +804,6 @@ func TestStreamingAutoReplan(t *testing.T) { clientAddresses[addr] = struct{}{} }, AfterRetryIteration: func(err error) { - if err != nil && !alreadyReplanned.Load() { retryErrorChan <- err <-turnOffReplanning diff --git a/pkg/crosscluster/physical/stream_ingestion_dist.go b/pkg/crosscluster/physical/stream_ingestion_dist.go index 37269066e602..853d5707b758 100644 --- a/pkg/crosscluster/physical/stream_ingestion_dist.go +++ b/pkg/crosscluster/physical/stream_ingestion_dist.go @@ -124,22 +124,25 @@ func startDistIngestion( return err } - err = ingestionJob.NoTxn().Update(ctx, func(txn isql.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { - // Persist the initial Stream Addresses to the jobs table before execution begins. - if len(planner.initialPartitionPgUrls) == 0 { - return jobs.MarkAsPermanentJobError(errors.AssertionFailedf( - "attempted to persist an empty list of partition connection uris")) - } - md.Progress.GetStreamIngest().PartitionConnUris = make([]string, len(planner.initialPartitionPgUrls)) - for i := range planner.initialPartitionPgUrls { - md.Progress.GetStreamIngest().PartitionConnUris[i] = planner.initialPartitionPgUrls[i].Serialize() + if planner.initialPartitionPgUrls[0].RoutingMode() != streamclient.RoutingModeGateway { + err = ingestionJob.NoTxn().Update(ctx, func(txn isql.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + // Persist the initial Stream Addresses to the jobs table before execution begins. + if len(planner.initialPartitionPgUrls) == 0 { + return jobs.MarkAsPermanentJobError(errors.AssertionFailedf( + "attempted to persist an empty list of partition connection uris")) + } + md.Progress.GetStreamIngest().PartitionConnUris = make([]string, len(planner.initialPartitionPgUrls)) + for i := range planner.initialPartitionPgUrls { + md.Progress.GetStreamIngest().PartitionConnUris[i] = planner.initialPartitionPgUrls[i].Serialize() + } + ju.UpdateProgress(md.Progress) + return nil + }) + if err != nil { + return errors.Wrap(err, "failed to update job progress") } - ju.UpdateProgress(md.Progress) - return nil - }) - if err != nil { - return errors.Wrap(err, "failed to update job progress") } + jobsprofiler.StorePlanDiagram(ctx, execCtx.ExecCfg().DistSQLSrv.Stopper, planner.initialPlan, execCtx.ExecCfg().InternalDB, ingestionJob.ID()) diff --git a/pkg/crosscluster/physical/stream_ingestion_job_test.go b/pkg/crosscluster/physical/stream_ingestion_job_test.go index 7d54e52349f3..31ef1ded3d16 100644 --- a/pkg/crosscluster/physical/stream_ingestion_job_test.go +++ b/pkg/crosscluster/physical/stream_ingestion_job_test.go @@ -9,6 +9,7 @@ import ( "context" gosql "database/sql" "fmt" + "net" "net/url" "testing" @@ -17,6 +18,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/crosscluster/producer" "github.com/cockroachdb/cockroach/pkg/crosscluster/replicationtestutils" "github.com/cockroachdb/cockroach/pkg/crosscluster/replicationutils" + "github.com/cockroachdb/cockroach/pkg/crosscluster/streamclient" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -31,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -129,26 +132,12 @@ func TestTenantStreamingFailback(t *testing.T) { sqlA := sqlutils.MakeSQLRunner(aDB) sqlB := sqlutils.MakeSQLRunner(bDB) - serverAURL, cleanupURLA := sqlutils.PGUrl(t, serverA.SQLAddr(), t.Name(), url.User(username.RootUser)) - defer cleanupURLA() - serverBURL, cleanupURLB := sqlutils.PGUrl(t, serverB.SQLAddr(), t.Name(), url.User(username.RootUser)) - defer cleanupURLB() - - for _, s := range []string{ - "SET CLUSTER SETTING kv.rangefeed.enabled = true", - "SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '200ms'", - "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'", - "SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '50ms'", - - "SET CLUSTER SETTING physical_replication.consumer.heartbeat_frequency = '1s'", - "SET CLUSTER SETTING physical_replication.consumer.job_checkpoint_frequency = '100ms'", - "SET CLUSTER SETTING physical_replication.consumer.minimum_flush_interval = '10ms'", - "SET CLUSTER SETTING physical_replication.consumer.failover_signal_poll_interval = '100ms'", - "SET CLUSTER SETTING spanconfig.reconciliation_job.checkpoint_interval = '100ms'", - } { - sqlA.Exec(t, s) - sqlB.Exec(t, s) - } + serverAURL := replicationtestutils.GetReplicationUri(t, serverA, serverB, serverutils.User(username.RootUser)) + serverBURL := replicationtestutils.GetReplicationUri(t, serverB, serverA, serverutils.User(username.RootUser)) + + replicationtestutils.ConfigureDefaultSettings(t, sqlA) + replicationtestutils.ConfigureDefaultSettings(t, sqlB) + compareAtTimetamp := func(ts string) { fingerprintQueryFmt := "SELECT fingerprint FROM [SHOW EXPERIMENTAL_FINGERPRINTS FROM TENANT %s] AS OF SYSTEM TIME %s" var fingerprintF int64 @@ -156,7 +145,6 @@ func TestTenantStreamingFailback(t *testing.T) { var fingerprintG int64 sqlB.QueryRow(t, fmt.Sprintf(fingerprintQueryFmt, "g", ts)).Scan(&fingerprintG) require.Equal(t, fingerprintF, fingerprintG, "fingerprint mismatch at %s", ts) - } // The overall test plan looks like: @@ -673,3 +661,59 @@ func waitUntilTenantServerStopped( return nil }) } + +func TestPhysicalReplicationGatewayRoute(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Create a blackhole so we can claim a port and black hole any connections + // routed there. + blackhole, err := net.Listen("tcp", "localhost:0") + require.NoError(t, err) + defer func() { + require.NoError(t, blackhole.Close()) + }() + + srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{ + DefaultTestTenant: base.TestControlsTenantsExplicitly, + Knobs: base.TestingKnobs{ + Streaming: &sql.StreamingTestingKnobs{ + OnGetSQLInstanceInfo: func(node *roachpb.NodeDescriptor) *roachpb.NodeDescriptor { + copy := *node + copy.SQLAddress = util.UnresolvedAddr{ + NetworkField: "tcp", + AddressField: blackhole.Addr().String(), + } + return © + }, + }, + }, + }) + defer srv.Stopper().Stop(context.Background()) + + systemDB := sqlutils.MakeSQLRunner(db) + + replicationtestutils.ConfigureDefaultSettings(t, systemDB) + + // Create the source tenant and start service + systemDB.Exec(t, "CREATE VIRTUAL CLUSTER source") + systemDB.Exec(t, "ALTER VIRTUAL CLUSTER source START SERVICE SHARED") + + serverURL, cleanup := srv.PGUrl(t) + defer cleanup() + + q := serverURL.Query() + q.Set(streamclient.RoutingModeKey, string(streamclient.RoutingModeGateway)) + serverURL.RawQuery = q.Encode() + + // Create the destination tenant by replicating the source cluster + systemDB.Exec(t, "CREATE VIRTUAL CLUSTER target FROM REPLICATION OF source ON $1", serverURL.String()) + + _, jobID := replicationtestutils.GetStreamJobIds(t, context.Background(), systemDB, "target") + + now := srv.Clock().Now() + replicationtestutils.WaitUntilReplicatedTime(t, now, systemDB, jobspb.JobID(jobID)) + + progress := jobutils.GetJobProgress(t, systemDB, jobspb.JobID(jobID)) + require.Empty(t, progress.Details.(*jobspb.Progress_StreamIngest).StreamIngest.PartitionConnUris) +} diff --git a/pkg/crosscluster/physical/stream_ingestion_planning_test.go b/pkg/crosscluster/physical/stream_ingestion_planning_test.go index 98ca5a9a1609..6dfc18c9782c 100644 --- a/pkg/crosscluster/physical/stream_ingestion_planning_test.go +++ b/pkg/crosscluster/physical/stream_ingestion_planning_test.go @@ -7,10 +7,10 @@ package physical import ( "context" - "net/url" "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/crosscluster/replicationtestutils" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -43,8 +43,7 @@ func TestCreateTenantFromReplicationUsingID(t *testing.T) { sqlA := sqlutils.MakeSQLRunner(aDB) sqlB := sqlutils.MakeSQLRunner(bDB) - serverAURL, cleanupURLA := sqlutils.PGUrl(t, serverA.SQLAddr(), t.Name(), url.User(username.RootUser)) - defer cleanupURLA() + serverAURL := replicationtestutils.GetReplicationUri(t, serverA, serverB, serverutils.User(username.RootUser)) verifyCreatedTenant := func(t *testing.T, db *sqlutils.SQLRunner, id int64, fn func()) { const query = "SELECT count(*), count(CASE WHEN id = $1 THEN 1 END) FROM system.tenants" diff --git a/pkg/crosscluster/producer/replication_manager.go b/pkg/crosscluster/producer/replication_manager.go index 781e5e0d8789..02ca6ce7001a 100644 --- a/pkg/crosscluster/producer/replication_manager.go +++ b/pkg/crosscluster/producer/replication_manager.go @@ -43,6 +43,7 @@ type replicationStreamManagerImpl struct { resolver resolver.SchemaResolver txn descs.Txn sessionID clusterunique.ID + knobs *sql.StreamingTestingKnobs } // StartReplicationStream implements streaming.ReplicationStreamManager interface. @@ -215,7 +216,7 @@ func (r *replicationStreamManagerImpl) PlanLogicalReplication( } } - spec, err := buildReplicationStreamSpec(ctx, r.evalCtx, tenID, false, spans, useStreaksInLDR.Get(&r.evalCtx.Settings.SV)) + spec, err := r.buildReplicationStreamSpec(ctx, r.evalCtx, tenID, false, spans, useStreaksInLDR.Get(&r.evalCtx.Settings.SV)) if err != nil { return nil, err } @@ -271,7 +272,7 @@ func (r *replicationStreamManagerImpl) GetPhysicalReplicationStreamSpec( if err := r.checkLicense(); err != nil { return nil, err } - return getPhysicalReplicationStreamSpec(ctx, r.evalCtx, r.txn, streamID) + return r.getPhysicalReplicationStreamSpec(ctx, r.evalCtx, r.txn, streamID) } // CompleteReplicationStream implements ReplicationStreamManager interface. @@ -290,7 +291,7 @@ func (r *replicationStreamManagerImpl) SetupSpanConfigsStream( if err := r.checkLicense(); err != nil { return nil, err } - return setupSpanConfigsStream(ctx, r.evalCtx, r.txn, tenantName) + return r.setupSpanConfigsStream(ctx, r.evalCtx, r.txn, tenantName) } func (r *replicationStreamManagerImpl) DebugGetProducerStatuses( @@ -351,7 +352,11 @@ func newReplicationStreamManagerWithPrivilegesCheck( privilege.REPLICATION); err != nil { return nil, err } - return &replicationStreamManagerImpl{evalCtx: evalCtx, txn: txn, sessionID: sessionID, resolver: sc}, nil + + execCfg := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig) + knobs := execCfg.StreamingTestingKnobs + + return &replicationStreamManagerImpl{evalCtx: evalCtx, txn: txn, sessionID: sessionID, resolver: sc, knobs: knobs}, nil } func (r *replicationStreamManagerImpl) checkLicense() error { diff --git a/pkg/crosscluster/producer/stream_lifetime.go b/pkg/crosscluster/producer/stream_lifetime.go index 7378995adcbf..d42438b1a4f5 100644 --- a/pkg/crosscluster/producer/stream_lifetime.go +++ b/pkg/crosscluster/producer/stream_lifetime.go @@ -257,7 +257,7 @@ func heartbeatReplicationStream( } // getPhysicalReplicationStreamSpec gets a replication stream specification for the specified stream. -func getPhysicalReplicationStreamSpec( +func (r *replicationStreamManagerImpl) getPhysicalReplicationStreamSpec( ctx context.Context, evalCtx *eval.Context, txn isql.Txn, streamID streampb.StreamID, ) (*streampb.ReplicationStreamSpec, error) { jobExecCtx := evalCtx.JobExecContext.(sql.JobExecContext) @@ -274,11 +274,10 @@ func getPhysicalReplicationStreamSpec( if j.Status() != jobs.StatusRunning { return nil, jobIsNotRunningError(jobID, j.Status(), "create stream spec") } - return buildReplicationStreamSpec(ctx, evalCtx, details.TenantID, false, details.Spans, true) - + return r.buildReplicationStreamSpec(ctx, evalCtx, details.TenantID, false, details.Spans, true) } -func buildReplicationStreamSpec( +func (r *replicationStreamManagerImpl) buildReplicationStreamSpec( ctx context.Context, evalCtx *eval.Context, tenantID roachpb.TenantID, @@ -326,6 +325,9 @@ func buildReplicationStreamSpec( if err != nil { return nil, err } + if r.knobs != nil && r.knobs.OnGetSQLInstanceInfo != nil { + nodeInfo = r.knobs.OnGetSQLInstanceInfo(nodeInfo) + } res.Partitions = append(res.Partitions, streampb.ReplicationStreamSpec_Partition{ NodeID: roachpb.NodeID(sp.SQLInstanceID), SQLAddress: nodeInfo.SQLAddress, @@ -379,7 +381,7 @@ func completeReplicationStream( }) } -func setupSpanConfigsStream( +func (r *replicationStreamManagerImpl) setupSpanConfigsStream( ctx context.Context, evalCtx *eval.Context, txn isql.Txn, tenantName roachpb.TenantName, ) (eval.ValueGenerator, error) { @@ -392,8 +394,8 @@ func setupSpanConfigsStream( execConfig := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig) spanConfigName := systemschema.SpanConfigurationsTableName - if knobs := execConfig.StreamingTestingKnobs; knobs != nil && knobs.MockSpanConfigTableName != nil { - spanConfigName = knobs.MockSpanConfigTableName + if r.knobs != nil && r.knobs.MockSpanConfigTableName != nil { + spanConfigName = r.knobs.MockSpanConfigTableName } if err := sql.DescsTxn(ctx, execConfig, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { diff --git a/pkg/crosscluster/replicationtestutils/BUILD.bazel b/pkg/crosscluster/replicationtestutils/BUILD.bazel index d4a2fc9d0e0b..ce61d37605c4 100644 --- a/pkg/crosscluster/replicationtestutils/BUILD.bazel +++ b/pkg/crosscluster/replicationtestutils/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "replication_helpers.go", "span_config_helpers.go", "testutils.go", + "uri_util.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/crosscluster/replicationtestutils", visibility = ["//visibility:public"], @@ -49,6 +50,7 @@ go_library( "//pkg/util", "//pkg/util/ctxgroup", "//pkg/util/hlc", + "//pkg/util/metamorphic", "//pkg/util/protoutil", "//pkg/util/randutil", "//pkg/util/retry", diff --git a/pkg/crosscluster/replicationtestutils/testutils.go b/pkg/crosscluster/replicationtestutils/testutils.go index e55f4ae4ff73..6b8d99e76df8 100644 --- a/pkg/crosscluster/replicationtestutils/testutils.go +++ b/pkg/crosscluster/replicationtestutils/testutils.go @@ -18,6 +18,7 @@ import ( apd "github.com/cockroachdb/apd/v3" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/crosscluster/replicationutils" + "github.com/cockroachdb/cockroach/pkg/crosscluster/streamclient" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" @@ -77,6 +78,8 @@ type TenantStreamingClustersArgs struct { NoMetamorphicExternalConnection bool ExternalIODir string + + RoutingMode streamclient.RoutingMode } var DefaultTenantStreamingClustersArgs = TenantStreamingClustersArgs{ @@ -93,11 +96,11 @@ var DefaultTenantStreamingClustersArgs = TenantStreamingClustersArgs{ `) }, SrcNumNodes: 1, - SrcClusterSettings: defaultSrcClusterSetting, + SrcClusterSettings: DefaultClusterSettings, DestTenantName: roachpb.TenantName("destination"), DestTenantID: roachpb.MustMakeTenantID(2), DestNumNodes: 1, - DestClusterSettings: defaultDestClusterSetting, + DestClusterSettings: DefaultClusterSettings, } type TenantStreamingClusters struct { @@ -451,6 +454,12 @@ func CreateMultiTenantStreamingCluster( cluster, url, cleanup := startC2CTestCluster(ctx, t, serverArgs, args.MultitenantSingleClusterNumNodes, args.MultiTenantSingleClusterTestRegions) + if args.RoutingMode != "" { + query := url.Query() + query.Set(streamclient.RoutingModeKey, string(args.RoutingMode)) + url.RawQuery = query.Encode() + } + rng, _ := randutil.NewPseudoRand() destNodeIdx := args.MultitenantSingleClusterNumNodes - 1 @@ -495,6 +504,11 @@ func CreateTenantStreamingClusters( g.GoCtx(func(ctx context.Context) error { // Start the source cluster. srcCluster, srcURL, srcCleanup = startC2CTestCluster(ctx, t, serverArgs, args.SrcNumNodes, args.SrcClusterTestRegions) + if args.RoutingMode != "" { + query := srcURL.Query() + query.Set(streamclient.RoutingModeKey, string(args.RoutingMode)) + srcURL.RawQuery = query.Encode() + } return nil }) @@ -626,33 +640,32 @@ func CreateScatteredTable(t *testing.T, c *TenantStreamingClusters, numNodes int }, timeout) } -var defaultSrcClusterSetting = map[string]string{ - `kv.rangefeed.enabled`: `true`, +var DefaultClusterSettings = map[string]string{ + `bulkio.stream_ingestion.failover_signal_poll_interval`: `'100ms'`, + `bulkio.stream_ingestion.minimum_flush_interval`: `'10ms'`, + `jobs.registry.interval.adopt`: `'1s'`, + `kv.bulk_io_write.small_write_size`: `'1'`, + `kv.closed_timestamp.side_transport_interval`: `'50ms'`, // Speed up the rangefeed. These were set by squinting at the settings set in // the changefeed integration tests. `kv.closed_timestamp.target_duration`: `'100ms'`, `kv.rangefeed.closed_timestamp_refresh_interval`: `'200ms'`, - `kv.closed_timestamp.side_transport_interval`: `'50ms'`, - // Large timeout makes test to not fail with unexpected timeout failures. - `stream_replication.stream_liveness_track_frequency`: `'2s'`, - `stream_replication.min_checkpoint_frequency`: `'1s'`, + `kv.rangefeed.enabled`: `true`, // Finer grain checkpoints to keep replicated time close to present. `physical_replication.producer.timestamp_granularity`: `'100ms'`, - // Make all AddSSTable operation to trigger AddSSTable events. - `kv.bulk_io_write.small_write_size`: `'1'`, - `jobs.registry.interval.adopt`: `'1s'`, // Speed up span reconciliation `spanconfig.reconciliation_job.checkpoint_interval`: `'100ms'`, + `stream_replication.consumer_heartbeat_frequency`: `'1s'`, + `stream_replication.job_checkpoint_frequency`: `'100ms'`, + `stream_replication.min_checkpoint_frequency`: `'1s'`, + // Large timeout makes test to not fail with unexpected timeout failures. + `stream_replication.stream_liveness_track_frequency`: `'2s'`, } -var defaultDestClusterSetting = map[string]string{ - `stream_replication.consumer_heartbeat_frequency`: `'1s'`, - `stream_replication.job_checkpoint_frequency`: `'100ms'`, - `bulkio.stream_ingestion.minimum_flush_interval`: `'10ms'`, - `bulkio.stream_ingestion.failover_signal_poll_interval`: `'100ms'`, - `jobs.registry.interval.adopt`: `'1s'`, - `spanconfig.reconciliation_job.checkpoint_interval`: `'100ms'`, - `kv.rangefeed.enabled`: `true`, +func ConfigureDefaultSettings(t *testing.T, sqlRunner *sqlutils.SQLRunner) { + for key, val := range DefaultClusterSettings { + sqlRunner.Exec(t, fmt.Sprintf("SET CLUSTER SETTING %s = %s;", key, val)) + } } func ConfigureClusterSettings(setting map[string]string) []string { diff --git a/pkg/crosscluster/replicationtestutils/uri_util.go b/pkg/crosscluster/replicationtestutils/uri_util.go new file mode 100644 index 000000000000..bc5e3ad1f7ca --- /dev/null +++ b/pkg/crosscluster/replicationtestutils/uri_util.go @@ -0,0 +1,50 @@ +package replicationtestutils + +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +import ( + "fmt" + "math/rand" + "net/url" + "testing" + + "github.com/cockroachdb/cockroach/pkg/crosscluster/streamclient" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/metamorphic" + "github.com/stretchr/testify/require" +) + +// TODO make the sql address unconnectable if useGatewayRoutingMode is true +var useGatewayRoutingMode = metamorphic.ConstantWithTestBool("stream-use-gateway-routing-mode", false) +var useExternalConnection = metamorphic.ConstantWithTestBool("stream-use-external-connection", true) + +func GetReplicationUri( + t *testing.T, + sourceCluster serverutils.ApplicationLayerInterface, + destCluster serverutils.ApplicationLayerInterface, + sourceConnOptions ...serverutils.SQLConnOption, +) url.URL { + sourceURI, cleanup := sourceCluster.PGUrl(t, sourceConnOptions...) + t.Cleanup(cleanup) + + if useGatewayRoutingMode { + query := sourceURI.Query() + query.Set(streamclient.RoutingModeKey, string(streamclient.RoutingModeGateway)) + sourceURI.RawQuery = query.Encode() + } + + if useExternalConnection { + conn := destCluster.SQLConn(t) + defer conn.Close() + + externalUri := url.URL{Scheme: "external", Host: fmt.Sprintf("replication-uri-%d", rand.Int63())} + _, err := conn.Exec(fmt.Sprintf("CREATE EXTERNAL CONNECTION '%s' AS '%s'", externalUri.Host, sourceURI.String())) + require.NoError(t, err) + return externalUri + } + + return sourceURI +} diff --git a/pkg/crosscluster/streamclient/partitioned_stream_client.go b/pkg/crosscluster/streamclient/partitioned_stream_client.go index a52c8b0bb707..bd9db8ed07ff 100644 --- a/pkg/crosscluster/streamclient/partitioned_stream_client.go +++ b/pkg/crosscluster/streamclient/partitioned_stream_client.go @@ -182,10 +182,17 @@ func (p *partitionedStreamClient) createTopology( SourceTenantID: spec.SourceTenantID, } for _, sp := range spec.Partitions { - nodeUri, err := p.clusterUri.ResolveNode(sp.SQLAddress) - if err != nil { - return Topology{}, err + var connUri ClusterUri + if p.clusterUri.RoutingMode() == RoutingModeGateway { + connUri = p.clusterUri + } else { + var err error + connUri, err = MakeClusterUriForNode(p.clusterUri, sp.SQLAddress) + if err != nil { + return Topology{}, err + } } + rawSpec, err := protoutil.Marshal(sp.SourcePartition) if err != nil { return Topology{}, err @@ -194,7 +201,7 @@ func (p *partitionedStreamClient) createTopology( ID: sp.NodeID.String(), SubscriptionToken: SubscriptionToken(rawSpec), SrcInstanceID: int(sp.NodeID), - ConnUri: nodeUri, + ConnUri: connUri, SrcLocality: sp.Locality, Spans: sp.SourcePartition.Spans, }) diff --git a/pkg/crosscluster/streamclient/uri.go b/pkg/crosscluster/streamclient/uri.go index a93f3c1d2833..7f4cc5430beb 100644 --- a/pkg/crosscluster/streamclient/uri.go +++ b/pkg/crosscluster/streamclient/uri.go @@ -89,6 +89,12 @@ func ParseClusterUri(uri string) (ClusterUri, error) { if !allowedConfigUriSchemes[url.Scheme] { return ClusterUri{}, errors.Newf("stream replication from scheme %q is unsupported", url.Scheme) } + if url.Query().Has(RoutingModeKey) { + mode := url.Query().Get(RoutingModeKey) + if mode != string(RoutingModeNode) && mode != string(RoutingModeGateway) { + return ClusterUri{}, errors.Newf("unknown crdb_route value %q", mode) + } + } return ClusterUri{uri: *url}, nil } @@ -105,14 +111,21 @@ func MakeTestClusterUri(url url.URL) ClusterUri { return ClusterUri{uri: url} } -func (sa *ClusterUri) ResolveNode(hostname util.UnresolvedAddr) (ClusterUri, error) { - host, port, err := net.SplitHostPort(hostname.AddressField) +// MakeClusterUriForNode creates a new ClusterUri with the node address set to the given +// address. MakeClusterUriForNode will return an error if the uri has routing mode +// gateway. +func MakeClusterUriForNode(uri ClusterUri, nodeAddress util.UnresolvedAddr) (ClusterUri, error) { + if uri.RoutingMode() == RoutingModeGateway { + return ClusterUri{}, errors.Newf("cannot set node address on gateway uri %s", uri.Redacted()) + } + + host, port, err := net.SplitHostPort(nodeAddress.AddressField) if err != nil { return ClusterUri{}, err } - copy := sa.uri - copy.Host = net.JoinHostPort(host, port) - return ClusterUri{uri: copy}, nil + copy := uri + copy.uri.Host = net.JoinHostPort(host, port) + return copy, nil } func (sa *ClusterUri) Serialize() string { @@ -137,3 +150,32 @@ func redactUrl(u url.URL) string { u.RawQuery = "redacted" return u.String() } + +const RoutingModeKey = "crdb_route" + +type RoutingMode string + +const ( + // routinModeNode is the default routing mode for LDR and PCR. The + // configuration uri is used to connect to the cluster and build a dist sql + // plan for the stream producers. The processors in the destination client + // then connect directly to the nodes described by the source cluster's plan. + RoutingModeNode RoutingMode = "node" + // routingModeGateway is a routing mode that replaces the default node + // routing mode. Processors in the source cluster will connect to the + // configured uri instead of the per-node uris returned by the source + // clusters plan. This allows for LDR and PCR to be used in situations where + // the source cluster nodes are not directly routable from the destination + // nodes. + RoutingModeGateway RoutingMode = "gateway" +) + +// RoutingMode returns the routing mode specified in the uri. If no routing +// mode is specified, the default routing mode is returned. The routing mode is +// validated by the ClusterUri constructor. +func (c *ClusterUri) RoutingMode() RoutingMode { + if key := c.uri.Query().Get(RoutingModeKey); key != "" { + return RoutingMode(key) + } + return RoutingModeNode +} diff --git a/pkg/crosscluster/streamclient/uri_test.go b/pkg/crosscluster/streamclient/uri_test.go index 36e330fc2ca6..d76139c6a825 100644 --- a/pkg/crosscluster/streamclient/uri_test.go +++ b/pkg/crosscluster/streamclient/uri_test.go @@ -50,6 +50,9 @@ func TestParseClusterUri(t *testing.T) { tests := []testCase{ {uri: "postgres://foo", err: ""}, {uri: "postgresql://foo", err: ""}, + {uri: "postgresql://foo?crdb_route=node", err: ""}, + {uri: "postgresql://foo?crdb_route=gateway", err: ""}, + {uri: "postgresql://foo?crdb_route=ohhno", err: "unknown crdb_route value \"ohhno\""}, {uri: "randomgen://foo", err: ""}, {uri: "external://foo", err: "external uri \"external://foo\" must be resolved before constructing a cluster uri"}, {uri: "ohhno://foo", err: "stream replication from scheme \"ohhno\" is unsupported"}, diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 6119f0899d21..a8028c2282b1 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1940,6 +1940,8 @@ type StreamingTestingKnobs struct { SpanConfigRangefeedCacheKnobs *rangefeedcache.TestingKnobs + OnGetSQLInstanceInfo func(cluster *roachpb.NodeDescriptor) *roachpb.NodeDescriptor + FailureRate uint32 } From 023dbc1b552bf4073a2ffb0e6e37ba8b02de4d50 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Tue, 14 Jan 2025 15:19:41 -0500 Subject: [PATCH 104/126] catalog/lease: WaitForInitialVersion should be version gated Previously, we could have mixed version configurations hang waiting for initial version of descriptors, when new objects are created. This was because prior releases of CRDB do not have logic to lease new descriptors automatically. This patch adds a version gate to enable the WaitForInitialVersion and the memo staleness optimizations once 25.1 is finalized. Fixes: #139015 Release note: None --- pkg/sql/catalog/lease/BUILD.bazel | 1 + pkg/sql/catalog/lease/lease.go | 4 +++- pkg/sql/opt/BUILD.bazel | 1 + pkg/sql/opt/metadata.go | 2 ++ 4 files changed, 7 insertions(+), 1 deletion(-) diff --git a/pkg/sql/catalog/lease/BUILD.bazel b/pkg/sql/catalog/lease/BUILD.bazel index 3cecbfd8b09e..98102f9ec8c5 100644 --- a/pkg/sql/catalog/lease/BUILD.bazel +++ b/pkg/sql/catalog/lease/BUILD.bazel @@ -20,6 +20,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/clusterversion", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/rangefeed", diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index d8899700fce6..7a42f5f86f19 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" @@ -313,7 +314,8 @@ func (m *Manager) WaitForInitialVersion( retryOpts retry.Options, regions regionliveness.CachedDatabaseRegions, ) error { - if !WaitForInitialVersion.Get(&m.settings.SV) { + if !WaitForInitialVersion.Get(&m.settings.SV) || + !m.storage.settings.Version.IsActive(ctx, clusterversion.V25_1) { return nil } wsTracker := startWaitStatsTracker(ctx) diff --git a/pkg/sql/opt/BUILD.bazel b/pkg/sql/opt/BUILD.bazel index 0cf6ff5f10da..0c0eefe0224e 100644 --- a/pkg/sql/opt/BUILD.bazel +++ b/pkg/sql/opt/BUILD.bazel @@ -24,6 +24,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/opt", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/server/telemetry", "//pkg/sql/catalog", "//pkg/sql/catalog/catpb", diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index ff4b7cae24c3..4cd3c11920d7 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -11,6 +11,7 @@ import ( "math/bits" "strings" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/multiregion" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" @@ -397,6 +398,7 @@ func (md *Metadata) CheckDependencies( // is sufficient. currentDigest := optCatalog.GetDependencyDigest() if evalCtx.SessionData().CatalogDigestStalenessCheckEnabled && + evalCtx.Settings.Version.IsActive(ctx, clusterversion.V25_1) && evalCtx.AsOfSystemTime == nil && !evalCtx.Txn.ReadTimestampFixed() && md.dependencyDigestEquals(¤tDigest) { From c9ef59a0ebf20f925fe196b517c1b1246b08a6ea Mon Sep 17 00:00:00 2001 From: Kevin Cao Date: Tue, 7 Jan 2025 18:45:24 -0500 Subject: [PATCH 105/126] sql: add CHECK EXTERNAL CONNECTION command This patch adds the `CHECK EXTERNAL CONNECTION` command and replaces the old `SHOW BACKUP CONNECTION` syntax. Epic: None Release note: None --- docs/generated/sql/bnf/BUILD.bazel | 2 + .../bnf/check_external_connection_stmt.bnf | 2 + docs/generated/sql/bnf/check_stmt.bnf | 2 + docs/generated/sql/bnf/preparable_stmt.bnf | 1 + docs/generated/sql/bnf/stmt_block.bnf | 20 ++ pkg/BUILD.bazel | 3 - pkg/backup/BUILD.bazel | 2 - pkg/backup/backup_planning.go | 7 +- pkg/backup/restore_planning.go | 3 +- pkg/backup/show.go | 49 +---- .../testdata/backup-restore/show_backup | 14 -- pkg/cloud/cloudcheck/BUILD.bazel | 30 --- pkg/cloud/cloudcheck/cloudcheck_stmt.go | 110 ---------- pkg/cloud/cloudprivilege/BUILD.bazel | 34 ---- pkg/cloud/cloudprivilege/privileges.go | 66 ------ pkg/cmd/docgen/diagrams.go | 2 +- pkg/gen/bnf.bzl | 2 + pkg/gen/diagrams.bzl | 2 + pkg/gen/docs.bzl | 2 + pkg/sql/BUILD.bazel | 3 + pkg/sql/check_external_connection.go | 188 ++++++++++++++++++ .../cloud_check_processor.go} | 75 ++++++- pkg/sql/export.go | 2 +- pkg/sql/importer/BUILD.bazel | 1 - pkg/sql/importer/import_planning.go | 3 +- pkg/sql/opaque.go | 3 + pkg/sql/parser/help_test.go | 3 + pkg/sql/parser/sql.y | 158 ++++++++------- pkg/sql/parser/testdata/backup_restore | 37 ---- .../parser/testdata/check_external_connection | 26 +++ pkg/sql/plan_columns.go | 2 + pkg/sql/sem/tree/BUILD.bazel | 1 + pkg/sql/sem/tree/check.go | 81 ++++++++ pkg/sql/sem/tree/show.go | 4 - pkg/sql/sem/tree/stmt.go | 10 + pkg/sql/walk.go | 1 + 36 files changed, 520 insertions(+), 431 deletions(-) create mode 100644 docs/generated/sql/bnf/check_external_connection_stmt.bnf create mode 100644 docs/generated/sql/bnf/check_stmt.bnf delete mode 100644 pkg/cloud/cloudcheck/BUILD.bazel delete mode 100644 pkg/cloud/cloudcheck/cloudcheck_stmt.go delete mode 100644 pkg/cloud/cloudprivilege/BUILD.bazel delete mode 100644 pkg/cloud/cloudprivilege/privileges.go create mode 100644 pkg/sql/check_external_connection.go rename pkg/{cloud/cloudcheck/cloudcheck_processor.go => sql/cloud_check_processor.go} (70%) create mode 100644 pkg/sql/parser/testdata/check_external_connection create mode 100644 pkg/sql/sem/tree/check.go diff --git a/docs/generated/sql/bnf/BUILD.bazel b/docs/generated/sql/bnf/BUILD.bazel index 79484692a94b..70b919705876 100644 --- a/docs/generated/sql/bnf/BUILD.bazel +++ b/docs/generated/sql/bnf/BUILD.bazel @@ -117,6 +117,8 @@ FILES = [ "create_trigger", "create_type", "create_view_stmt", + "check_stmt", + "check_external_connection_stmt", "deallocate_stmt", "declare_cursor_stmt", "default_value_column_level", diff --git a/docs/generated/sql/bnf/check_external_connection_stmt.bnf b/docs/generated/sql/bnf/check_external_connection_stmt.bnf new file mode 100644 index 000000000000..650dc5f60a09 --- /dev/null +++ b/docs/generated/sql/bnf/check_external_connection_stmt.bnf @@ -0,0 +1,2 @@ +check_external_connection_stmt ::= + 'CHECK' 'EXTERNAL' 'CONNECTION' string_or_placeholder opt_with_check_external_connection_options_list diff --git a/docs/generated/sql/bnf/check_stmt.bnf b/docs/generated/sql/bnf/check_stmt.bnf new file mode 100644 index 000000000000..6f3e5486e7d3 --- /dev/null +++ b/docs/generated/sql/bnf/check_stmt.bnf @@ -0,0 +1,2 @@ +check_stmt ::= + check_external_connection_stmt diff --git a/docs/generated/sql/bnf/preparable_stmt.bnf b/docs/generated/sql/bnf/preparable_stmt.bnf index 69e0e9e7f119..8080d3da63ca 100644 --- a/docs/generated/sql/bnf/preparable_stmt.bnf +++ b/docs/generated/sql/bnf/preparable_stmt.bnf @@ -3,6 +3,7 @@ preparable_stmt ::= | backup_stmt | cancel_stmt | create_stmt + | check_stmt | delete_stmt | drop_stmt | explain_stmt diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 15cf61aa499f..c316a7a9f241 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -43,6 +43,7 @@ preparable_stmt ::= | backup_stmt | cancel_stmt | create_stmt + | check_stmt | delete_stmt | drop_stmt | explain_stmt @@ -212,6 +213,9 @@ create_stmt ::= | create_logical_replication_stream_stmt | create_schedule_stmt +check_stmt ::= + check_external_connection_stmt + delete_stmt ::= opt_with_clause 'DELETE' opt_batch_clause 'FROM' table_expr_opt_alias_idx opt_using_clause opt_where_clause opt_sort_clause opt_limit_clause returning_clause @@ -635,6 +639,9 @@ create_schedule_stmt ::= create_schedule_for_changefeed_stmt | create_schedule_for_backup_stmt +check_external_connection_stmt ::= + 'CHECK' 'EXTERNAL' 'CONNECTION' string_or_placeholder opt_with_check_external_connection_options_list + opt_with_clause ::= with_clause | @@ -1909,6 +1916,11 @@ create_schedule_for_changefeed_stmt ::= create_schedule_for_backup_stmt ::= 'CREATE' 'SCHEDULE' schedule_label_spec 'FOR' 'BACKUP' opt_backup_targets 'INTO' string_or_placeholder_opt_list opt_with_backup_options cron_expr opt_full_backup_clause opt_with_schedule_options +opt_with_check_external_connection_options_list ::= + 'WITH' check_external_connection_options_list + | 'WITH' 'OPTIONS' '(' check_external_connection_options_list ')' + | + with_clause ::= 'WITH' cte_list | 'WITH' 'RECURSIVE' cte_list @@ -2728,6 +2740,9 @@ opt_full_backup_clause ::= | 'FULL' 'BACKUP' 'ALWAYS' | +check_external_connection_options_list ::= + ( check_external_connection_options ) ( ( ',' check_external_connection_options ) )* + cte_list ::= ( common_table_expr ) ( ( ',' common_table_expr ) )* @@ -3365,6 +3380,11 @@ logical_replication_create_table_options ::= | 'UNIDIRECTIONAL' | 'BIDIRECTIONAL' 'ON' string_or_placeholder +check_external_connection_options ::= + 'TRANSFER' '=' string_or_placeholder + | 'TIME' '=' string_or_placeholder + | 'CONCURRENTLY' '=' a_expr + common_table_expr ::= table_alias_name opt_col_def_list_no_types 'AS' materialize_clause '(' preparable_stmt ')' diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index cd55c8a4bd4a..3169ffcc1f59 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1037,10 +1037,7 @@ GO_TARGETS = [ "//pkg/cloud/amazon:amazon_test", "//pkg/cloud/azure:azure", "//pkg/cloud/azure:azure_test", - "//pkg/cloud/cloudcheck:cloudcheck", "//pkg/cloud/cloudpb:cloudpb", - "//pkg/cloud/cloudprivilege:cloudprivilege", - "//pkg/cloud/cloudprivilege:privileges", "//pkg/cloud/cloudtestutils:cloudtestutils", "//pkg/cloud/externalconn/connectionpb:connectionpb", "//pkg/cloud/externalconn/providers:providers", diff --git a/pkg/backup/BUILD.bazel b/pkg/backup/BUILD.bazel index ed87db3bdd64..0a1f25b5f753 100644 --- a/pkg/backup/BUILD.bazel +++ b/pkg/backup/BUILD.bazel @@ -51,9 +51,7 @@ go_library( "//pkg/ccl/multiregionccl", "//pkg/ccl/storageccl", "//pkg/cloud", - "//pkg/cloud/cloudcheck", "//pkg/cloud/cloudpb", - "//pkg/cloud/cloudprivilege", "//pkg/clusterversion", "//pkg/featureflag", "//pkg/jobs", diff --git a/pkg/backup/backup_planning.go b/pkg/backup/backup_planning.go index fdb5dede77be..2678a7570c60 100644 --- a/pkg/backup/backup_planning.go +++ b/pkg/backup/backup_planning.go @@ -14,7 +14,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/backup/backupbase" "github.com/cockroachdb/cockroach/pkg/backup/backupresolver" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/cloud/cloudprivilege" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -271,7 +270,7 @@ func checkPrivilegesForBackup( pgcode.InsufficientPrivilege, "only users with the admin role or the BACKUP system privilege are allowed to perform full cluster backups") } - return cloudprivilege.CheckDestinationPrivileges(ctx, p, to) + return sql.CheckDestinationPrivileges(ctx, p, to) } } @@ -328,7 +327,7 @@ func checkPrivilegesForBackup( // If a user has the BACKUP privilege on the target databases or tables we can // move on to checking the destination URIs. if hasRequiredBackupPrivileges { - return cloudprivilege.CheckDestinationPrivileges(ctx, p, to) + return sql.CheckDestinationPrivileges(ctx, p, to) } // The following checks are to maintain compatability with pre-22.2 privilege @@ -362,7 +361,7 @@ func checkPrivilegesForBackup( } } - return cloudprivilege.CheckDestinationPrivileges(ctx, p, to) + return sql.CheckDestinationPrivileges(ctx, p, to) } func backupTypeCheck( diff --git a/pkg/backup/restore_planning.go b/pkg/backup/restore_planning.go index b66a1e4e84a6..f61aefb45ccc 100644 --- a/pkg/backup/restore_planning.go +++ b/pkg/backup/restore_planning.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/cloud/cloudprivilege" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -1436,7 +1435,7 @@ func restorePlanHook( func checkRestoreDestinationPrivileges( ctx context.Context, p sql.PlanHookState, from []string, ) error { - return cloudprivilege.CheckDestinationPrivileges(ctx, p, from) + return sql.CheckDestinationPrivileges(ctx, p, from) } // checkRestorePrivilegesOnDatabase check that the user has adequate privileges diff --git a/pkg/backup/show.go b/pkg/backup/show.go index 67d4e8613599..af8e4ad2a765 100644 --- a/pkg/backup/show.go +++ b/pkg/backup/show.go @@ -21,8 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/backup/backuputils" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/cloud/cloudcheck" - "github.com/cockroachdb/cockroach/pkg/cloud/cloudprivilege" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -45,7 +43,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -208,9 +205,6 @@ func showBackupTypeCheck( ); err != nil { return false, nil, err } - if backup.Details == tree.BackupConnectionTest { - return true, cloudcheck.Header, nil - } infoReader := getBackupInfoReader(p, backup) return true, infoReader.header(), nil } @@ -225,45 +219,6 @@ func showBackupPlanHook( } exprEval := p.ExprEvaluator("SHOW BACKUP") - // TODO(dt): find move this to its own hook. - if showStmt.Details == tree.BackupConnectionTest { - loc, err := exprEval.String(ctx, showStmt.Path) - if err != nil { - return nil, nil, false, err - } - var params cloudcheck.Params - if showStmt.Options.CheckConnectionTransferSize != nil { - transferSizeStr, err := exprEval.String(ctx, showStmt.Options.CheckConnectionTransferSize) - if err != nil { - return nil, nil, false, err - } - parsed, err := humanizeutil.ParseBytes(transferSizeStr) - if err != nil { - return nil, nil, false, err - } - params.TransferSize = parsed - } - if showStmt.Options.CheckConnectionDuration != nil { - durationStr, err := exprEval.String(ctx, showStmt.Options.CheckConnectionDuration) - if err != nil { - return nil, nil, false, err - } - parsed, err := time.ParseDuration(durationStr) - if err != nil { - return nil, nil, false, err - } - params.MinDuration = parsed - } - if showStmt.Options.CheckConnectionConcurrency != nil { - concurrency, err := exprEval.Int(ctx, showStmt.Options.CheckConnectionConcurrency) - if err != nil { - return nil, nil, false, err - } - params.Concurrency = concurrency - } - return cloudcheck.ShowCloudStorageTestPlanHook(ctx, p, loc, params) - } - if showStmt.Path == nil && showStmt.InCollection != nil { collection, err := exprEval.StringArray( ctx, tree.Exprs(showStmt.InCollection), @@ -319,7 +274,7 @@ func showBackupPlanHook( "https://www.cockroachlabs.com/docs/stable/show-backup.html")) } - if err := cloudprivilege.CheckDestinationPrivileges(ctx, p, dest); err != nil { + if err := sql.CheckDestinationPrivileges(ctx, p, dest); err != nil { return err } @@ -1517,7 +1472,7 @@ func showBackupsInCollectionPlanHook( ctx context.Context, collection []string, showStmt *tree.ShowBackup, p sql.PlanHookState, ) (sql.PlanHookRowFn, colinfo.ResultColumns, bool, error) { - if err := cloudprivilege.CheckDestinationPrivileges(ctx, p, collection); err != nil { + if err := sql.CheckDestinationPrivileges(ctx, p, collection); err != nil { return nil, nil, false, err } diff --git a/pkg/backup/testdata/backup-restore/show_backup b/pkg/backup/testdata/backup-restore/show_backup index 3fc2afc2e0fa..7e2ace9ba171 100644 --- a/pkg/backup/testdata/backup-restore/show_backup +++ b/pkg/backup/testdata/backup-restore/show_backup @@ -63,17 +63,3 @@ query-sql regex=No\sproblems\sfound! SELECT * FROM [SHOW BACKUP VALIDATE FROM 'valid-22.2-with-job' IN 'nodelocal://1/']; ---- true - -query-sql -SELECT node, locality, ok, error, can_delete FROM [SHOW BACKUP CONNECTION 'nodelocal://1/conn-test' WITH TRANSFER='1'] ORDER BY node; ----- -1 region=eu-central-1,availability-zone=eu-central-1 true true -2 region=eu-north-1,availability-zone=eu-north-1 true true -3 region=us-east-1,availability-zone=us-east1 true true - -query-sql -SELECT node, locality, ok, error, can_delete FROM [SHOW BACKUP CONNECTION 'nodelocal://1/conn-test' WITH TRANSFER='1', TIME = '1ms'] ORDER BY node; ----- -1 region=eu-central-1,availability-zone=eu-central-1 true true -2 region=eu-north-1,availability-zone=eu-north-1 true true -3 region=us-east-1,availability-zone=us-east1 true true diff --git a/pkg/cloud/cloudcheck/BUILD.bazel b/pkg/cloud/cloudcheck/BUILD.bazel deleted file mode 100644 index bfbb7e188c2f..000000000000 --- a/pkg/cloud/cloudcheck/BUILD.bazel +++ /dev/null @@ -1,30 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "cloudcheck", - srcs = [ - "cloudcheck_processor.go", - "cloudcheck_stmt.go", - ], - importpath = "github.com/cockroachdb/cockroach/pkg/cloud/cloudcheck", - visibility = ["//visibility:public"], - deps = [ - "//pkg/cloud", - "//pkg/cloud/cloudprivilege", - "//pkg/security/username", - "//pkg/sql", - "//pkg/sql/catalog/colinfo", - "//pkg/sql/execinfra", - "//pkg/sql/execinfrapb", - "//pkg/sql/physicalplan", - "//pkg/sql/rowenc", - "//pkg/sql/rowexec", - "//pkg/sql/sem/tree", - "//pkg/sql/types", - "//pkg/util/ctxgroup", - "//pkg/util/humanizeutil", - "//pkg/util/timeutil", - "//pkg/util/tracing", - "@com_github_cockroachdb_errors//:errors", - ], -) diff --git a/pkg/cloud/cloudcheck/cloudcheck_stmt.go b/pkg/cloud/cloudcheck/cloudcheck_stmt.go deleted file mode 100644 index e16e7f0af8f7..000000000000 --- a/pkg/cloud/cloudcheck/cloudcheck_stmt.go +++ /dev/null @@ -1,110 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -package cloudcheck - -import ( - "context" - "time" - - "github.com/cockroachdb/cockroach/pkg/cloud/cloudprivilege" - "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" - "github.com/cockroachdb/cockroach/pkg/util/tracing" - "github.com/cockroachdb/errors" -) - -var Header = colinfo.ResultColumns{ - {Name: "node", Typ: types.Int}, - {Name: "locality", Typ: types.String}, - {Name: "ok", Typ: types.Bool}, - {Name: "error", Typ: types.String}, - {Name: "transferred", Typ: types.String}, - {Name: "read_speed", Typ: types.String}, - {Name: "write_speed", Typ: types.String}, - {Name: "can_delete", Typ: types.Bool}, -} - -// Params configure the behavior of a cloud storage check. -type Params = execinfrapb.CloudStorageTestSpec_Params - -// ShowCloudStorageTestPlanHook is currently called by showBackup hook but -// should be extended to be a standalone plan instead. -func ShowCloudStorageTestPlanHook( - ctx context.Context, p sql.PlanHookState, location string, params Params, -) (sql.PlanHookRowFn, colinfo.ResultColumns, bool, error) { - - if err := cloudprivilege.CheckDestinationPrivileges(ctx, p, []string{location}); err != nil { - return nil, nil, false, err - } - - fn := func(ctx context.Context, resultsCh chan<- tree.Datums) error { - ctx, span := tracing.ChildSpan(ctx, "ShowCloudStorageTestPlanHook") - defer span.Finish() - - store, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, location, p.User()) - if err != nil { - return errors.Wrapf(err, "connect to external storage") - } - defer store.Close() - - dsp := p.DistSQLPlanner() - evalCtx := p.ExtendedEvalContext() - planCtx, sqlInstanceIDs, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, p.ExecCfg()) - if err != nil { - return err - } - plan := planCtx.NewPhysicalPlan() - corePlacement := make([]physicalplan.ProcessorCorePlacement, len(sqlInstanceIDs)) - spec := &execinfrapb.CloudStorageTestSpec{Location: location, Params: params} - for i := range sqlInstanceIDs { - corePlacement[i].SQLInstanceID = sqlInstanceIDs[i] - corePlacement[i].Core.CloudStorageTest = spec - } - plan.AddNoInputStage(corePlacement, execinfrapb.PostProcessSpec{}, flowTypes, execinfrapb.Ordering{}) - - plan.PlanToStreamColMap = make([]int, len(flowTypes)) - for i := range plan.PlanToStreamColMap { - plan.PlanToStreamColMap[i] = i - } - sql.FinalizePlan(ctx, planCtx, plan) - - rateFromDatums := func(bytes tree.Datum, nanos tree.Datum) string { - return string(humanizeutil.DataRate(int64(tree.MustBeDInt(bytes)), time.Duration(tree.MustBeDInt(nanos)))) - } - rowResultWriter := sql.NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { - // collapse the two pairs of bytes+time to a single string rate each. - res := make(tree.Datums, len(row)-1) - copy(res[:4], row[:4]) - res[4] = tree.NewDString(string(humanizeutil.IBytes(int64(tree.MustBeDInt(row[6]))))) - res[5] = tree.NewDString(rateFromDatums(row[4], row[5])) - res[6] = tree.NewDString(rateFromDatums(row[6], row[7])) - res[7] = row[8] - resultsCh <- res - return nil - }) - - recv := sql.MakeDistSQLReceiver( - ctx, - rowResultWriter, - tree.Rows, - nil, /* rangeCache */ - nil, /* txn - the flow does not read or write the database */ - nil, /* clockUpdater */ - evalCtx.Tracing, - ) - defer recv.Release() - - evalCtxCopy := *evalCtx - dsp.Run(ctx, planCtx, nil, plan, recv, &evalCtxCopy, nil /* finishedSetupFn */) - return rowResultWriter.Err() - } - return fn, Header, false, nil -} diff --git a/pkg/cloud/cloudprivilege/BUILD.bazel b/pkg/cloud/cloudprivilege/BUILD.bazel deleted file mode 100644 index 5a6efa26e4d0..000000000000 --- a/pkg/cloud/cloudprivilege/BUILD.bazel +++ /dev/null @@ -1,34 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "privileges", - srcs = ["privileges.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/cloud/privileges", - visibility = ["//visibility:public"], - deps = [ - "//pkg/cloud", - "//pkg/cloud/cloudpb", - "//pkg/clusterversion", - "//pkg/sql", - "//pkg/sql/pgwire/pgcode", - "//pkg/sql/pgwire/pgerror", - "//pkg/sql/privilege", - "//pkg/sql/syntheticprivilege", - ], -) - -go_library( - name = "cloudprivilege", - srcs = ["privileges.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/cloud/cloudprivilege", - visibility = ["//visibility:public"], - deps = [ - "//pkg/cloud", - "//pkg/cloud/cloudpb", - "//pkg/sql", - "//pkg/sql/pgwire/pgcode", - "//pkg/sql/pgwire/pgerror", - "//pkg/sql/privilege", - "//pkg/sql/syntheticprivilege", - ], -) diff --git a/pkg/cloud/cloudprivilege/privileges.go b/pkg/cloud/cloudprivilege/privileges.go deleted file mode 100644 index baa95bf137a3..000000000000 --- a/pkg/cloud/cloudprivilege/privileges.go +++ /dev/null @@ -1,66 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -package cloudprivilege - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/cloud/cloudpb" - "github.com/cockroachdb/cockroach/pkg/sql" - "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/syntheticprivilege" -) - -// CheckDestinationPrivileges iterates over the External Storage URIs and -// ensures the user has adequate privileges to use each of them. -func CheckDestinationPrivileges(ctx context.Context, p sql.PlanHookState, to []string) error { - isAdmin, err := p.UserHasAdminRole(ctx, p.User()) - if err != nil { - return err - } - if isAdmin { - return nil - } - - // Check destination specific privileges. - for _, uri := range to { - conf, err := cloud.ExternalStorageConfFromURI(uri, p.User()) - if err != nil { - return err - } - - // Check if the destination requires the user to be an admin or have the - // `EXTERNALIOIMPLICITACCESS` privilege. - requiresImplicitAccess := !conf.AccessIsWithExplicitAuth() - hasImplicitAccessPrivilege, privErr := - p.HasPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.EXTERNALIOIMPLICITACCESS, p.User()) - if privErr != nil { - return privErr - } - if requiresImplicitAccess && !(p.ExecCfg().ExternalIODirConfig.EnableNonAdminImplicitAndArbitraryOutbound || hasImplicitAccessPrivilege) { - return pgerror.Newf( - pgcode.InsufficientPrivilege, - "only users with the admin role or the EXTERNALIOIMPLICITACCESS system privilege are allowed to access the specified %s URI", - conf.Provider.String()) - } - - // If the resource being used is an External Connection, check that the user - // has adequate privileges. - if conf.Provider == cloudpb.ExternalStorageProvider_external { - ecPrivilege := &syntheticprivilege.ExternalConnectionPrivilege{ - ConnectionName: conf.ExternalConnectionConfig.Name, - } - if err := p.CheckPrivilege(ctx, ecPrivilege, privilege.USAGE); err != nil { - return err - } - } - } - - return nil -} diff --git a/pkg/cmd/docgen/diagrams.go b/pkg/cmd/docgen/diagrams.go index 9f21fa121de8..ec72a5d52664 100644 --- a/pkg/cmd/docgen/diagrams.go +++ b/pkg/cmd/docgen/diagrams.go @@ -1484,7 +1484,7 @@ var specs = []stmtSpec{ { name: "show_backup", stmt: "show_backup_stmt", - inline: []string{"opt_with_options", "show_backup_details", "opt_with_show_backup_options", "opt_with_show_backup_connection_options_list", "show_backup_connection_options_list", "show_backup_options_list"}, + inline: []string{"opt_with_options", "show_backup_details", "opt_with_show_backup_options", "show_backup_options_list"}, replace: map[string]string{ "'BACKUPS' 'IN' string_or_placeholder_opt_list": "'BACKUPS' 'IN' collectionURI", "'BACKUP' string_or_placeholder 'IN' string_or_placeholder_opt_list": "'BACKUP' subdirectory 'IN' collectionURI", diff --git a/pkg/gen/bnf.bzl b/pkg/gen/bnf.bzl index fe847040399f..7f759adfe4cd 100644 --- a/pkg/gen/bnf.bzl +++ b/pkg/gen/bnf.bzl @@ -81,6 +81,8 @@ BNF_SRCS = [ "//docs/generated/sql/bnf:cancel_session.bnf", "//docs/generated/sql/bnf:cancel_stmt.bnf", "//docs/generated/sql/bnf:check_column_level.bnf", + "//docs/generated/sql/bnf:check_external_connection_stmt.bnf", + "//docs/generated/sql/bnf:check_stmt.bnf", "//docs/generated/sql/bnf:check_table_level.bnf", "//docs/generated/sql/bnf:close_cursor_stmt.bnf", "//docs/generated/sql/bnf:col_qualification.bnf", diff --git a/pkg/gen/diagrams.bzl b/pkg/gen/diagrams.bzl index f22730ddcdd6..41f6d8e539d7 100644 --- a/pkg/gen/diagrams.bzl +++ b/pkg/gen/diagrams.bzl @@ -80,7 +80,9 @@ DIAGRAMS_SRCS = [ "//docs/generated/sql/bnf:cancel_job.html", "//docs/generated/sql/bnf:cancel_query.html", "//docs/generated/sql/bnf:cancel_session.html", + "//docs/generated/sql/bnf:check.html", "//docs/generated/sql/bnf:check_column_level.html", + "//docs/generated/sql/bnf:check_external_connection.html", "//docs/generated/sql/bnf:check_table_level.html", "//docs/generated/sql/bnf:close_cursor.html", "//docs/generated/sql/bnf:col_qualification.html", diff --git a/pkg/gen/docs.bzl b/pkg/gen/docs.bzl index 59fb44877617..62c1ab7ebf0e 100644 --- a/pkg/gen/docs.bzl +++ b/pkg/gen/docs.bzl @@ -94,6 +94,8 @@ DOCS_SRCS = [ "//docs/generated/sql/bnf:cancel_session.bnf", "//docs/generated/sql/bnf:cancel_stmt.bnf", "//docs/generated/sql/bnf:check_column_level.bnf", + "//docs/generated/sql/bnf:check_external_connection_stmt.bnf", + "//docs/generated/sql/bnf:check_stmt.bnf", "//docs/generated/sql/bnf:check_table_level.bnf", "//docs/generated/sql/bnf:close_cursor_stmt.bnf", "//docs/generated/sql/bnf:col_qualification.bnf", diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 44aa291b06c3..404d7039ca6f 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -33,7 +33,9 @@ go_library( "cancel_queries.go", "cancel_sessions.go", "check.go", + "check_external_connection.go", "closed_session_cache.go", + "cloud_check_processor.go", "comment.go", "comment_on_column.go", "comment_on_constraint.go", @@ -303,6 +305,7 @@ go_library( "//pkg/base", "//pkg/build", "//pkg/cloud", + "//pkg/cloud/cloudpb", "//pkg/cloud/externalconn", "//pkg/clusterversion", "//pkg/col/coldata", diff --git a/pkg/sql/check_external_connection.go b/pkg/sql/check_external_connection.go new file mode 100644 index 000000000000..44b56ad3c895 --- /dev/null +++ b/pkg/sql/check_external_connection.go @@ -0,0 +1,188 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package sql + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/errors" +) + +func (p *planner) CheckExternalConnection( + _ context.Context, n *tree.CheckExternalConnection, +) (planNode, error) { + return &checkExternalConnectionNode{node: n, columns: CloudCheckHeader}, nil +} + +type checkExternalConnectionNode struct { + zeroInputPlanNode + node *tree.CheckExternalConnection + loc string + params CloudCheckParams + rows chan tree.Datums + row tree.Datums + columns colinfo.ResultColumns +} + +var _ planNode = &checkExternalConnectionNode{} + +func (n *checkExternalConnectionNode) startExec(params runParams) error { + if err := n.parseParams(params); err != nil { + return err + } + if err := CheckDestinationPrivileges( + params.ctx, + params.p, + []string{n.loc}, + ); err != nil { + return err + } + + ctx, span := tracing.ChildSpan(params.ctx, "CheckExternalConnection") + defer span.Finish() + + store, err := params.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, n.loc, params.p.User()) + if err != nil { + return errors.Wrap(err, "connect to external storage") + } + defer store.Close() + + dsp := params.p.DistSQLPlanner() + evalCtx := params.extendedEvalCtx + planCtx, sqlInstanceIDs, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, params.ExecCfg()) + if err != nil { + return err + } + plan := planCtx.NewPhysicalPlan() + corePlacement := make([]physicalplan.ProcessorCorePlacement, len(sqlInstanceIDs)) + spec := &execinfrapb.CloudStorageTestSpec{Location: n.loc, Params: n.params} + for i := range sqlInstanceIDs { + corePlacement[i].SQLInstanceID = sqlInstanceIDs[i] + corePlacement[i].Core.CloudStorageTest = spec + } + plan.AddNoInputStage( + corePlacement, + execinfrapb.PostProcessSpec{}, + cloudCheckFlowTypes, + execinfrapb.Ordering{}, + ) + plan.PlanToStreamColMap = make([]int, len(cloudCheckFlowTypes)) + for i := range plan.PlanToStreamColMap { + plan.PlanToStreamColMap[i] = i + } + FinalizePlan(ctx, planCtx, plan) + + rateFromDatums := func(bytes tree.Datum, nanos tree.Datum) string { + return string(humanizeutil.DataRate( + int64(tree.MustBeDInt(bytes)), + time.Duration(tree.MustBeDInt(nanos)), + )) + } + n.rows = make(chan tree.Datums, int64(len(sqlInstanceIDs))*n.params.Concurrency) + rowWriter := NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { + // collapse the two pairs of bytes+time to a single string rate each. + res := make(tree.Datums, len(row)-1) + copy(res[:4], row[:4]) + res[4] = tree.NewDString(string(humanizeutil.IBytes(int64(tree.MustBeDInt(row[6]))))) + res[5] = tree.NewDString(rateFromDatums(row[4], row[5])) + res[6] = tree.NewDString(rateFromDatums(row[6], row[7])) + res[7] = row[8] + n.rows <- res + return nil + }) + + go func() { + recv := MakeDistSQLReceiver( + ctx, + rowWriter, + tree.Rows, + nil, /* rangeCache */ + nil, /* txn - the flow does not read or write the database */ + nil, /* clockUpdater */ + evalCtx.Tracing, + ) + defer recv.Release() + defer close(n.rows) + + evalCtxCopy := *evalCtx + dsp.Run(ctx, planCtx, nil, plan, recv, &evalCtxCopy, nil /* finishedSetupFn */) + }() + return nil +} + +func (n *checkExternalConnectionNode) Next(params runParams) (bool, error) { + if n.rows == nil { + return false, nil + } + select { + case <-params.ctx.Done(): + return false, params.ctx.Err() + case row, more := <-n.rows: + if !more { + n.rows = nil + return false, nil + } + n.row = row + return true, nil + } +} + +func (n *checkExternalConnectionNode) Values() tree.Datums { + return n.row +} + +func (n *checkExternalConnectionNode) Close(_ context.Context) { + if n.rows != nil { + close(n.rows) + n.rows = nil + } +} + +func (n *checkExternalConnectionNode) parseParams(params runParams) error { + exprEval := params.p.ExprEvaluator("CHECK EXTERNAL CONNECTION") + loc, err := exprEval.String(params.ctx, n.node.URI) + if err != nil { + return err + } + n.loc = loc + if n.node.Options.TransferSize != nil { + transferSizeStr, err := exprEval.String(params.ctx, n.node.Options.TransferSize) + if err != nil { + return err + } + parsed, err := humanizeutil.ParseBytes(transferSizeStr) + if err != nil { + return err + } + n.params.TransferSize = parsed + } + if n.node.Options.Duration != nil { + durationStr, err := exprEval.String(params.ctx, n.node.Options.Duration) + if err != nil { + return err + } + parsed, err := time.ParseDuration(durationStr) + if err != nil { + return err + } + n.params.MinDuration = parsed + } + if n.node.Options.Concurrency != nil { + concurrency, err := exprEval.Int(params.ctx, n.node.Options.Concurrency) + if err != nil { + return err + } + n.params.Concurrency = concurrency + } + return nil +} diff --git a/pkg/cloud/cloudcheck/cloudcheck_processor.go b/pkg/sql/cloud_check_processor.go similarity index 70% rename from pkg/cloud/cloudcheck/cloudcheck_processor.go rename to pkg/sql/cloud_check_processor.go index d003aaa6e267..664e0d461f7c 100644 --- a/pkg/cloud/cloudcheck/cloudcheck_processor.go +++ b/pkg/sql/cloud_check_processor.go @@ -3,7 +3,7 @@ // Use of this software is governed by the CockroachDB Software License // included in the /LICENSE file. -package cloudcheck +package sql import ( "context" @@ -13,18 +13,37 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/cloud/cloudpb" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "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/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) +var CloudCheckHeader = colinfo.ResultColumns{ + {Name: "node", Typ: types.Int}, + {Name: "locality", Typ: types.String}, + {Name: "ok", Typ: types.Bool}, + {Name: "error", Typ: types.String}, + {Name: "transferred", Typ: types.String}, + {Name: "read_speed", Typ: types.String}, + {Name: "write_speed", Typ: types.String}, + {Name: "can_delete", Typ: types.Bool}, +} + +type CloudCheckParams = execinfrapb.CloudStorageTestSpec_Params + type result struct { ok bool error string @@ -35,7 +54,7 @@ type result struct { canDelete bool } -var flowTypes = []*types.T{ +var cloudCheckFlowTypes = []*types.T{ types.Int, types.String, // node and locality types.Bool, types.String, // ok and error types.Int, types.Int, // read bytes/nanos @@ -48,7 +67,7 @@ func checkURI( opener cloud.ExternalStorageFromURIFactory, location string, username username.SQLUsername, - params Params, + params CloudCheckParams, ) result { ctxDone := ctx.Done() @@ -178,7 +197,7 @@ func newCloudCheckProcessor( post *execinfrapb.PostProcessSpec, ) (execinfra.Processor, error) { p := &proc{spec: spec} - if err := p.Init(ctx, p, post, flowTypes, flowCtx, processorID, nil /* memMonitor */, execinfra.ProcStateOpts{}); err != nil { + if err := p.Init(ctx, p, post, cloudCheckFlowTypes, flowCtx, processorID, nil /* memMonitor */, execinfra.ProcStateOpts{}); err != nil { return nil, err } return p, nil @@ -246,6 +265,54 @@ func (p *proc) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) { } } +// CheckDestinationPrivileges iterates over the External Storage URIs and +// ensures the user has adequate privileges to use each of them. +func CheckDestinationPrivileges(ctx context.Context, p PlanHookState, to []string) error { + isAdmin, err := p.UserHasAdminRole(ctx, p.User()) + if err != nil { + return err + } + if isAdmin { + return nil + } + + // Check destination specific privileges. + for _, uri := range to { + conf, err := cloud.ExternalStorageConfFromURI(uri, p.User()) + if err != nil { + return err + } + + // Check if the destination requires the user to be an admin or have the + // `EXTERNALIOIMPLICITACCESS` privilege. + requiresImplicitAccess := !conf.AccessIsWithExplicitAuth() + hasImplicitAccessPrivilege, privErr := + p.HasPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.EXTERNALIOIMPLICITACCESS, p.User()) + if privErr != nil { + return privErr + } + if requiresImplicitAccess && !(p.ExecCfg().ExternalIODirConfig.EnableNonAdminImplicitAndArbitraryOutbound || hasImplicitAccessPrivilege) { + return pgerror.Newf( + pgcode.InsufficientPrivilege, + "only users with the admin role or the EXTERNALIOIMPLICITACCESS system privilege are allowed to access the specified %s URI", + conf.Provider.String()) + } + + // If the resource being used is an External Connection, check that the user + // has adequate privileges. + if conf.Provider == cloudpb.ExternalStorageProvider_external { + ecPrivilege := &syntheticprivilege.ExternalConnectionPrivilege{ + ConnectionName: conf.ExternalConnectionConfig.Name, + } + if err := p.CheckPrivilege(ctx, ecPrivilege, privilege.USAGE); err != nil { + return err + } + } + } + + return nil +} + func init() { rowexec.NewCloudStorageTestProcessor = newCloudCheckProcessor } diff --git a/pkg/sql/export.go b/pkg/sql/export.go index 0b208b2e1596..8ece86584a1d 100644 --- a/pkg/sql/export.go +++ b/pkg/sql/export.go @@ -146,7 +146,7 @@ func (ef *execFactory) ConstructExport( panic(err) } // TODO(adityamaru): Ideally we'd use - // `cloudprivilege.CheckDestinationPrivileges privileges here, but because of + // `sql.CheckDestinationPrivileges privileges here, but because of // a ciruclar dependancy with `pkg/sql` this is not possible. Consider moving // this file into `pkg/sql/importer` to get around this. hasExternalIOImplicitAccess := ef.planner.CheckPrivilege( diff --git a/pkg/sql/importer/BUILD.bazel b/pkg/sql/importer/BUILD.bazel index d7806121d656..48a9485496c3 100644 --- a/pkg/sql/importer/BUILD.bazel +++ b/pkg/sql/importer/BUILD.bazel @@ -32,7 +32,6 @@ go_library( deps = [ "//pkg/base", "//pkg/cloud", - "//pkg/cloud/cloudprivilege", "//pkg/clusterversion", "//pkg/col/coldata", "//pkg/crosscluster", diff --git a/pkg/sql/importer/import_planning.go b/pkg/sql/importer/import_planning.go index 45c5b1d3f630..c6b66bd4b330 100644 --- a/pkg/sql/importer/import_planning.go +++ b/pkg/sql/importer/import_planning.go @@ -17,7 +17,6 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/cloud/cloudprivilege" "github.com/cockroachdb/cockroach/pkg/docs" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -412,7 +411,7 @@ func importPlanHook( } return nil, nil, false, err } - if err := cloudprivilege.CheckDestinationPrivileges(ctx, p, []string{file}); err != nil { + if err := sql.CheckDestinationPrivileges(ctx, p, []string{file}); err != nil { return nil, nil, false, err } } diff --git a/pkg/sql/opaque.go b/pkg/sql/opaque.go index 10e1361bc79d..47e3e413d654 100644 --- a/pkg/sql/opaque.go +++ b/pkg/sql/opaque.go @@ -189,6 +189,8 @@ func planOpaque(ctx context.Context, p *planner, stmt tree.Statement) (planNode, return p.CreateExternalConnection(ctx, n) case *tree.CreateTenant: return p.CreateTenantNode(ctx, n) + case *tree.CheckExternalConnection: + return p.CheckExternalConnection(ctx, n) case *tree.DropExternalConnection: return p.DropExternalConnection(ctx, n) case *tree.Deallocate: @@ -450,6 +452,7 @@ func init() { &tree.ScheduledBackup{}, &tree.CreateTenantFromReplication{}, &tree.CreateLogicalReplicationStream{}, + &tree.CheckExternalConnection{}, } { typ := optbuilder.OpaqueReadOnly if tree.CanModifySchema(stmt) { diff --git a/pkg/sql/parser/help_test.go b/pkg/sql/parser/help_test.go index f05fe36cbd52..95feb2417854 100644 --- a/pkg/sql/parser/help_test.go +++ b/pkg/sql/parser/help_test.go @@ -215,6 +215,9 @@ func TestContextualHelp(t *testing.T) { {`CREATE SCHEMA IF NOT ??`, `CREATE SCHEMA`}, {`CREATE SCHEMA bli ??`, `CREATE SCHEMA`}, + {`CHECK ??`, `CHECK`}, + {`CHECK EXTERNAL CONNECTION ??`, `CHECK EXTERNAL CONNECTION`}, + {`DELETE FROM ??`, `DELETE`}, {`DELETE FROM blah ??`, `DELETE`}, {`DELETE FROM blah WHERE ??`, `DELETE`}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index c6678d937f41..bd84cf4e21a1 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -747,6 +747,9 @@ func (u *sqlSymUnion) showBackupDetails() tree.ShowBackupDetails { func (u *sqlSymUnion) showBackupOptions() *tree.ShowBackupOptions { return u.val.(*tree.ShowBackupOptions) } +func (u *sqlSymUnion) checkExternalConnectionOptions() *tree.CheckExternalConnectionOptions { + return u.val.(*tree.CheckExternalConnectionOptions) +} func (u *sqlSymUnion) restoreOptions() *tree.RestoreOptions { return u.val.(*tree.RestoreOptions) } @@ -1269,13 +1272,17 @@ func (u *sqlSymUnion) doBlockOption() tree.DoBlockOption { %type create_trigger_stmt %type create_policy_stmt -%type logical_replication_resources, logical_replication_resources_list +%type check_stmt +%type check_external_connection_stmt + +%type logical_replication_resources logical_replication_resources_list %type <*tree.LogicalReplicationOptions> opt_logical_replication_options logical_replication_options logical_replication_options_list opt_logical_replication_create_table_options logical_replication_create_table_options logical_replication_create_table_options_list %type create_stats_stmt %type <*tree.CreateStatsOptions> opt_create_stats_options %type <*tree.CreateStatsOptions> create_stats_option_list %type <*tree.CreateStatsOptions> create_stats_option +%type <*tree.CheckExternalConnectionOptions> opt_with_check_external_connection_options_list check_external_connection_options_list check_external_connection_options %type create_type_stmt %type delete_stmt @@ -1423,7 +1430,6 @@ func (u *sqlSymUnion) doBlockOption() tree.DoBlockOption { %type reindex_stmt -%type <[]string> opt_incremental %type kv_option %type <[]tree.KVOption> kv_option_list opt_with_options var_set_list opt_with_schedule_options %type <*tree.BackupOptions> opt_with_backup_options backup_options backup_options_list @@ -1431,7 +1437,7 @@ func (u *sqlSymUnion) doBlockOption() tree.DoBlockOption { %type <*tree.TenantReplicationOptions> opt_with_replication_options replication_options replication_options_list %type show_backup_details %type <*tree.ShowJobOptions> show_job_options show_job_options_list -%type <*tree.ShowBackupOptions> opt_with_show_backup_options show_backup_options show_backup_options_list show_backup_connection_options opt_with_show_backup_connection_options_list show_backup_connection_options_list +%type <*tree.ShowBackupOptions> opt_with_show_backup_options show_backup_options show_backup_options_list %type <*tree.CopyOptions> opt_with_copy_options copy_options copy_options_list copy_generic_options copy_generic_options_list %type import_format %type storage_parameter_key @@ -3821,6 +3827,64 @@ create_external_connection_stmt: } | CREATE EXTERNAL CONNECTION error // SHOW HELP: CREATE EXTERNAL CONNECTION +// %Help: CHECK EXTERNAL CONNECTION - check the status of an external connection +// %Category: Misc +// %Text: +// CREATE EXTERNAL CONNECTION [WITH ] +// +// Uri: +// Uri for the external connection. +check_external_connection_stmt: + CHECK EXTERNAL CONNECTION string_or_placeholder opt_with_check_external_connection_options_list + { + $$.val = &tree.CheckExternalConnection{ + URI: $4.expr(), + Options: *($5.checkExternalConnectionOptions()), + } + } + | CHECK EXTERNAL CONNECTION error // SHOW HELP: CHECK EXTERNAL CONNECTION + +opt_with_check_external_connection_options_list: + WITH check_external_connection_options_list + { + $$.val = $2.checkExternalConnectionOptions() + } +| WITH OPTIONS '(' check_external_connection_options_list ')' + { + $$.val = $4.checkExternalConnectionOptions() + } +| /* EMPTY */ + { + $$.val = &tree.CheckExternalConnectionOptions{} + } + +check_external_connection_options_list: + // Require at least one option + check_external_connection_options + { + $$.val = $1.checkExternalConnectionOptions() + } +| check_external_connection_options_list ',' check_external_connection_options + { + if err := $1.checkExternalConnectionOptions().CombineWith($3.checkExternalConnectionOptions()); err != nil { + return setErr(sqllex, err) + } + } + +check_external_connection_options: + TRANSFER '=' string_or_placeholder + { + $$.val = &tree.CheckExternalConnectionOptions{TransferSize: $3.expr()} + } + | TIME '=' string_or_placeholder + { + $$.val = &tree.CheckExternalConnectionOptions{Duration: $3.expr()} + } + | CONCURRENTLY '=' a_expr + { + $$.val = &tree.CheckExternalConnectionOptions{Concurrency: $3.expr()} + } + // %Help: DROP EXTERNAL CONNECTION - drop an existing external connection // %Category: Misc // %Text: @@ -3831,9 +3895,9 @@ create_external_connection_stmt: drop_external_connection_stmt: DROP EXTERNAL CONNECTION string_or_placeholder { - $$.val = &tree.DropExternalConnection{ - ConnectionLabel: $4.expr(), - } + $$.val = &tree.DropExternalConnection{ + ConnectionLabel: $4.expr(), + } } | DROP EXTERNAL CONNECTION error // SHOW HELP: DROP EXTERNAL CONNECTION @@ -4158,16 +4222,6 @@ string_or_placeholder_list: $$.val = append($1.exprs(), $3.expr()) } -opt_incremental: - INCREMENTAL FROM string_or_placeholder_list - { - $$.val = $3.exprs() - } -| /* EMPTY */ - { - $$.val = tree.Exprs(nil) - } - kv_option: name '=' string_or_placeholder { @@ -4652,11 +4706,19 @@ create_stmt: | create_unsupported {} | CREATE error // SHOW HELP: CREATE +// %Help: CHECK +// %Category: Group +// %Text: +// CHECK EXTERNAL CONNECTION +check_stmt: + check_external_connection_stmt // EXTEND WITH HELP: CHECK EXTERNAL CONNECTION +| CHECK error // SHOW HELP: CHECK + // %Help: CREATE LOGICAL REPLICATION STREAM - create a new logical replication stream // %Category: Experimental // %Text: -// CREATE LOGICAL REPLICATION STREAM -// FROM +// CREATE LOGICAL REPLICATION STREAM +// FROM
// ON 'stream_uri' // INTO
// [WITH @@ -4685,7 +4747,7 @@ create_logical_replication_stream_stmt: PGURL: $8.expr(), CreateTable: true, Options: *$9.logicalReplicationOptions(), - } + } } | CREATE LOGICAL REPLICATION STREAM error // SHOW HELP: CREATE LOGICAL REPLICATION STREAM @@ -4722,7 +4784,7 @@ logical_replication_resources_list: { $$.val = tree.LogicalReplicationResources{ Tables: append($1.logicalReplicationResources().Tables, $3.unresolvedObjectName().ToUnresolvedName()), - } + } } // Optional logical replication options. @@ -4794,7 +4856,7 @@ logical_replication_options: | DEFAULT FUNCTION '=' string_or_placeholder { $$.val = &tree.LogicalReplicationOptions{DefaultFunction: $4.expr()} - } + } | FUNCTION db_object_name FOR TABLE db_object_name { $$.val = &tree.LogicalReplicationOptions{UserFunctions: map[tree.UnresolvedName]tree.RoutineName{*$5.unresolvedObjectName().ToUnresolvedName():$2.unresolvedObjectName().ToRoutineName()}} @@ -4805,7 +4867,7 @@ logical_replication_options: } | SKIP SCHEMA CHECK { - $$.val = &tree.LogicalReplicationOptions{SkipSchemaCheck: tree.MakeDBool(true)} + $$.val = &tree.LogicalReplicationOptions{SkipSchemaCheck: tree.MakeDBool(true)} } | LABEL '=' string_or_placeholder { @@ -6760,6 +6822,7 @@ preparable_stmt: | backup_stmt // EXTEND WITH HELP: BACKUP | cancel_stmt // help texts in sub-rule | create_stmt // help texts in sub-rule +| check_stmt // help texts in sub-rule | delete_stmt // EXTEND WITH HELP: DELETE | drop_stmt // help texts in sub-rule | explain_stmt // EXTEND WITH HELP: EXPLAIN @@ -8675,13 +8738,11 @@ show_backup_stmt: setErr(sqllex, errors.New("The `SHOW BACKUP VALIDATE` syntax without the `IN` keyword is no longer supported. Please use `SHOW BACKUP VALIDATE FROM IN `.")) return helpWith(sqllex, "SHOW BACKUP") } -| SHOW BACKUP CONNECTION string_or_placeholder opt_with_show_backup_connection_options_list error +| SHOW BACKUP CONNECTION string_or_placeholder error { - $$.val = &tree.ShowBackup{ - Details: tree.BackupConnectionTest, - Path: $4.expr(), - Options: *$5.showBackupOptions(), - } + /* SKIP DOC */ + setErr(sqllex, errors.New("The `SHOW BACKUP CONNECTION` syntax is no longer supported. Please use `CHECK EXTERNAL CONNECTION`.")) + return helpWith(sqllex, "CHECK EXTERNAL CONNECTION") } | SHOW BACKUP error // SHOW HELP: SHOW BACKUP @@ -8784,47 +8845,6 @@ show_backup_options: $$.val = &tree.ShowBackupOptions{DebugMetadataSST: true} } -opt_with_show_backup_connection_options_list: - WITH show_backup_connection_options_list - { - $$.val = $2.showBackupOptions() - } -| WITH OPTIONS '(' show_backup_connection_options_list ')' - { - $$.val = $4.showBackupOptions() - } -| /* EMPTY */ - { - $$.val = &tree.ShowBackupOptions{} - } - -show_backup_connection_options_list: - // Require at least one option - show_backup_connection_options - { - $$.val = $1.showBackupOptions() - } -| show_backup_connection_options_list ',' show_backup_connection_options - { - if err := $1.showBackupOptions().CombineWith($3.showBackupOptions()); err != nil { - return setErr(sqllex, err) - } - } - -show_backup_connection_options: - TRANSFER '=' string_or_placeholder - { - $$.val = &tree.ShowBackupOptions{CheckConnectionTransferSize: $3.expr()} - } - | TIME '=' string_or_placeholder - { - $$.val = &tree.ShowBackupOptions{CheckConnectionDuration: $3.expr()} - } - | CONCURRENTLY '=' a_expr - { - $$.val = &tree.ShowBackupOptions{CheckConnectionConcurrency: $3.expr()} - } - // %Help: SHOW CLUSTER SETTING - display cluster settings // %Category: Cfg // %Text: diff --git a/pkg/sql/parser/testdata/backup_restore b/pkg/sql/parser/testdata/backup_restore index c383d37e51a7..3a528191dd91 100644 --- a/pkg/sql/parser/testdata/backup_restore +++ b/pkg/sql/parser/testdata/backup_restore @@ -141,33 +141,6 @@ SHOW BACKUP FROM '_' IN ('_', '_') WITH OPTIONS (incremental_location = ('_', '_ SHOW BACKUP FROM 'latest' IN ('*****', '*****') WITH OPTIONS (incremental_location = ('*****', '*****'), kms = ('*****', '*****')) -- identifiers removed SHOW BACKUP FROM 'latest' IN ('bar', 'bar1') WITH OPTIONS (incremental_location = ('hi', 'hello'), kms = ('foo', 'bar')) -- passwords exposed -parse -SHOW BACKUP CONNECTION bar' ----- -SHOW BACKUP CONNECTION '*****' -- normalized! -SHOW BACKUP CONNECTION ('*****') -- fully parenthesized -SHOW BACKUP CONNECTION '_' -- literals removed -SHOW BACKUP CONNECTION '*****' -- identifiers removed -SHOW BACKUP CONNECTION 'bar' -- passwords exposed - -parse -SHOW BACKUP CONNECTION 'bar' WITH TRANSFER = '1KiB', TIME = '1h', CONCURRENTLY = 3 ----- -SHOW BACKUP CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '1KiB', TIME = '1h') -- normalized! -SHOW BACKUP CONNECTION ('*****') WITH OPTIONS (CONCURRENTLY = (3), TRANSFER = ('1KiB'), TIME = ('1h')) -- fully parenthesized -SHOW BACKUP CONNECTION '_' WITH OPTIONS (CONCURRENTLY = _, TRANSFER = '_', TIME = '_') -- literals removed -SHOW BACKUP CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '1KiB', TIME = '1h') -- identifiers removed -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '1KiB', TIME = '1h') -- passwords exposed - -parse -SHOW BACKUP CONNECTION 'bar' WITH TRANSFER = $1, CONCURRENTLY = $2, TIME = $3 ----- -SHOW BACKUP CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- normalized! -SHOW BACKUP CONNECTION ('*****') WITH OPTIONS (CONCURRENTLY = ($2), TRANSFER = ($1), TIME = ($3)) -- fully parenthesized -SHOW BACKUP CONNECTION '_' WITH OPTIONS (CONCURRENTLY = $1, TRANSFER = $1, TIME = $1) -- literals removed -SHOW BACKUP CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- identifiers removed -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- passwords exposed - parse SHOW BACKUPS IN 'bar' ---- @@ -930,16 +903,6 @@ BACKUP INTO LATEST IN '_' -- literals removed BACKUP INTO LATEST IN '*****' -- identifiers removed BACKUP INTO LATEST IN 'unlogged' -- passwords exposed -# Regression test for https://github.com/cockroachdb/cockroach/issues/110411. -parse -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (TIME = '1h') ----- -SHOW BACKUP CONNECTION '*****' WITH OPTIONS (TIME = '1h') -- normalized! -SHOW BACKUP CONNECTION ('*****') WITH OPTIONS (TIME = ('1h')) -- fully parenthesized -SHOW BACKUP CONNECTION '_' WITH OPTIONS (TIME = '_') -- literals removed -SHOW BACKUP CONNECTION '*****' WITH OPTIONS (TIME = '1h') -- identifiers removed -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (TIME = '1h') -- passwords exposed - # Removal of old BACKUP/RESTORE syntax error BACKUP TO 'foo' diff --git a/pkg/sql/parser/testdata/check_external_connection b/pkg/sql/parser/testdata/check_external_connection new file mode 100644 index 000000000000..9bcd8e318a3b --- /dev/null +++ b/pkg/sql/parser/testdata/check_external_connection @@ -0,0 +1,26 @@ +parse +CHECK EXTERNAL CONNECTION 'foo' +---- +CHECK EXTERNAL CONNECTION '*****' -- normalized! +CHECK EXTERNAL CONNECTION ('*****') -- fully parenthesized +CHECK EXTERNAL CONNECTION '_' -- literals removed +CHECK EXTERNAL CONNECTION '*****' -- identifiers removed +CHECK EXTERNAL CONNECTION 'foo' -- passwords exposed + +parse +CHECK EXTERNAL CONNECTION 'foo' WITH CONCURRENTLY = 3, TRANSFER = '2 MiB', TIME = '1s' +---- +CHECK EXTERNAL CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '2 MiB', TIME = '1s') -- normalized! +CHECK EXTERNAL CONNECTION ('*****') WITH OPTIONS (CONCURRENTLY = (3), TRANSFER = ('2 MiB'), TIME = ('1s')) -- fully parenthesized +CHECK EXTERNAL CONNECTION '_' WITH OPTIONS (CONCURRENTLY = _, TRANSFER = '_', TIME = '_') -- literals removed +CHECK EXTERNAL CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '2 MiB', TIME = '1s') -- identifiers removed +CHECK EXTERNAL CONNECTION 'foo' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '2 MiB', TIME = '1s') -- passwords exposed + +parse +CHECK EXTERNAL CONNECTION 'foo' WITH TRANSFER = $1, CONCURRENTLY = $2, TIME = $3 +---- +CHECK EXTERNAL CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- normalized! +CHECK EXTERNAL CONNECTION ('*****') WITH OPTIONS (CONCURRENTLY = ($2), TRANSFER = ($1), TIME = ($3)) -- fully parenthesized +CHECK EXTERNAL CONNECTION '_' WITH OPTIONS (CONCURRENTLY = $1, TRANSFER = $1, TIME = $1) -- literals removed +CHECK EXTERNAL CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- identifiers removed +CHECK EXTERNAL CONNECTION 'foo' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- passwords exposed diff --git a/pkg/sql/plan_columns.go b/pkg/sql/plan_columns.go index 5a3d4a04ae2e..004a69b3d40d 100644 --- a/pkg/sql/plan_columns.go +++ b/pkg/sql/plan_columns.go @@ -97,6 +97,8 @@ func getPlanColumns(plan planNode, mut bool) colinfo.ResultColumns { return n.columns case *callNode: return n.getResultColumns() + case *checkExternalConnectionNode: + return n.columns // Nodes with a fixed schema. case *scrubNode: diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 64d8e47fb08c..fb211ca30260 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -26,6 +26,7 @@ go_library( "batch.go", "call.go", "changefeed.go", + "check.go", "col_name.go", "comment_on_column.go", "comment_on_constraint.go", diff --git a/pkg/sql/sem/tree/check.go b/pkg/sql/sem/tree/check.go new file mode 100644 index 000000000000..36e28b00bfe5 --- /dev/null +++ b/pkg/sql/sem/tree/check.go @@ -0,0 +1,81 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package tree + +type CheckExternalConnection struct { + URI Expr + Options CheckExternalConnectionOptions +} + +func (c *CheckExternalConnection) Format(ctx *FmtCtx) { + ctx.WriteString("CHECK EXTERNAL CONNECTION ") + ctx.FormatURI(c.URI) + if !c.Options.IsDefault() { + ctx.WriteString(" WITH OPTIONS (") + ctx.FormatNode(&c.Options) + ctx.WriteString(")") + } +} + +var _ Statement = &CheckExternalConnection{} + +type CheckExternalConnectionOptions struct { + TransferSize Expr + Duration Expr + Concurrency Expr +} + +var _ NodeFormatter = &CheckExternalConnectionOptions{} + +func (c *CheckExternalConnectionOptions) Format(ctx *FmtCtx) { + var addSep bool + maybeAddSep := func() { + if addSep { + ctx.WriteString(", ") + } + addSep = true + } + if c.Concurrency != nil { + maybeAddSep() + ctx.WriteString("CONCURRENTLY = ") + ctx.FormatNode(c.Concurrency) + } + if c.TransferSize != nil { + maybeAddSep() + ctx.WriteString("TRANSFER = ") + ctx.FormatNode(c.TransferSize) + } + if c.Duration != nil { + maybeAddSep() + ctx.WriteString("TIME = ") + ctx.FormatNode(c.Duration) + } +} + +func (c *CheckExternalConnectionOptions) CombineWith(other *CheckExternalConnectionOptions) error { + var err error + c.TransferSize, err = combineExpr(c.TransferSize, other.TransferSize, "transfer") + if err != nil { + return err + } + c.Concurrency, err = combineExpr(c.Concurrency, other.Concurrency, "concurrently") + if err != nil { + return err + } + c.Duration, err = combineExpr(c.Duration, other.Duration, "time") + if err != nil { + return err + } + return nil +} + +// IsDefault returns true if this options object is empty. +func (c *CheckExternalConnectionOptions) IsDefault() bool { + options := CheckExternalConnectionOptions{} + return c.Duration == options.Duration && + c.TransferSize == options.TransferSize && + c.Concurrency == options.Concurrency +} diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index 1bdc0ffb8c5b..a5d0fee5ee62 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -86,8 +86,6 @@ const ( // BackupValidateDetails identifies a SHOW BACKUP VALIDATION // statement. BackupValidateDetails - // BackupConnectionTest identifies a SHOW BACKUP CONNECTION statement - BackupConnectionTest ) // TODO (msbutler): 22.2 after removing old style show backup syntax, rename @@ -120,8 +118,6 @@ func (node *ShowBackup) Format(ctx *FmtCtx) { ctx.WriteString("FILES ") case BackupSchemaDetails: ctx.WriteString("SCHEMAS ") - case BackupConnectionTest: - ctx.WriteString("CONNECTION ") } if node.From { diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index f62e6325e2dd..5502102c9bb1 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -937,6 +937,15 @@ func (*CreateExternalConnection) StatementType() StatementType { return TypeDDL // StatementTag returns a short string identifying the type of statement. func (*CreateExternalConnection) StatementTag() string { return "CREATE EXTERNAL CONNECTION" } +// StatementReturnType implements the Statement interface. +func (*CheckExternalConnection) StatementReturnType() StatementReturnType { return Rows } + +// StatementType implements the Statement interface. +func (*CheckExternalConnection) StatementType() StatementType { return TypeDML } + +// StatementTag returns a short string identifying the type of statement. +func (*CheckExternalConnection) StatementTag() string { return "CHECK EXTERNAL CONNECTION" } + // StatementReturnType implements the Statement interface. func (*CreateTenant) StatementReturnType() StatementReturnType { return Ack } @@ -2542,6 +2551,7 @@ func (n *Explain) String() string { return AsString( func (n *ExplainAnalyze) String() string { return AsString(n) } func (n *Export) String() string { return AsString(n) } func (n *CreateExternalConnection) String() string { return AsString(n) } +func (n *CheckExternalConnection) String() string { return AsString(n) } func (n *DropExternalConnection) String() string { return AsString(n) } func (n *FetchCursor) String() string { return AsString(n) } func (n *Grant) String() string { return AsString(n) } diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index e754dc886bde..c2187b980ae0 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -394,6 +394,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&createTypeNode{}): "create type", reflect.TypeOf(&CreateRoleNode{}): "create user/role", reflect.TypeOf(&createViewNode{}): "create view", + reflect.TypeOf(&checkExternalConnectionNode{}): "check external connection", reflect.TypeOf(&delayedNode{}): "virtual table", reflect.TypeOf(&deleteNode{}): "delete", reflect.TypeOf(&deleteRangeNode{}): "delete range", From 4850f7fbea9811a9a3046d054e164c7d188f9b53 Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Fri, 10 Jan 2025 13:54:39 -0800 Subject: [PATCH 106/126] sql: change CPutAllowingIfNotExists with nil expValue to CPut CPutAllowingIfNotExists with empty expValue is equivalent to CPut with empty expValue, so change a few spots to use regular CPut. This almost gets rid of CPutAllowingIfNotExists entirely, but there is at least one spot in backfill (introduced in #138707) that needs to allow for both a non-empty expValue and non-existence of the KV. Also drop "(expecting does not exist)" from CPut tracing, as CPut with empty expValue is now overwhelmingly the most common use of CPut. And this matches the tracing in #138707. Epic: None Release note: None --- .../partitioning_implicit_read_committed | 2 +- pkg/sql/catalog/bootstrap/kv_writer.go | 2 +- pkg/sql/opt/exec/execbuilder/testdata/cascade | 12 ++--- .../exec/execbuilder/testdata/partial_index | 54 +++++++++---------- .../testdata/secondary_index_column_families | 20 +++---- pkg/sql/opt/exec/execbuilder/testdata/upsert | 2 +- pkg/sql/row/updater.go | 8 +-- pkg/sql/testdata/index_mutations/merging | 2 +- 8 files changed, 51 insertions(+), 51 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit_read_committed b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit_read_committed index 7941d3ee1bad..838a3ae43958 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit_read_committed +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit_read_committed @@ -446,7 +446,7 @@ CPut /Table/110/1/"@"/1/0 -> nil (tombstone) CPut /Table/110/1/"\x80"/1/0 -> nil (tombstone) CPut /Table/110/1/"\xc0"/1/0 -> nil (tombstone) CPut /Table/110/2/"\xa0"/6/0 -> /BYTES/0x89 -CPut /Table/110/2/"\xa0"/4/0 -> /BYTES/0x8a (expecting does not exist) +CPut /Table/110/2/"\xa0"/4/0 -> /BYTES/0x8a CPut /Table/110/2/" "/4/0 -> nil (tombstone) CPut /Table/110/2/"@"/4/0 -> nil (tombstone) CPut /Table/110/2/"\x80"/4/0 -> nil (tombstone) diff --git a/pkg/sql/catalog/bootstrap/kv_writer.go b/pkg/sql/catalog/bootstrap/kv_writer.go index f9f0cf3ccede..4d310ea16d56 100644 --- a/pkg/sql/catalog/bootstrap/kv_writer.go +++ b/pkg/sql/catalog/bootstrap/kv_writer.go @@ -97,7 +97,7 @@ func (w KVWriter) Insert( if kvTrace { log.VEventf(ctx, 2, "CPut %s -> %s", kv.Key, kv.Value) } - b.CPutAllowingIfNotExists(kv.Key, &kv.Value, nil /* expValue */) + b.CPut(kv.Key, &kv.Value, nil /* expValue */) } return nil } diff --git a/pkg/sql/opt/exec/execbuilder/testdata/cascade b/pkg/sql/opt/exec/execbuilder/testdata/cascade index 1caa37fc52bd..9aa082ef2c7d 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/cascade +++ b/pkg/sql/opt/exec/execbuilder/testdata/cascade @@ -175,10 +175,10 @@ Del /Table/112/1/"updated"/0 CPut /Table/112/1/"updated2"/0 -> /TUPLE/ executing cascade for constraint b1_update_cascade_fkey Del /Table/113/2/"updated"/0 -CPut /Table/113/2/"updated2"/0 -> /BYTES/0x1262312d706b310001 (expecting does not exist) +CPut /Table/113/2/"updated2"/0 -> /BYTES/0x1262312d706b310001 executing cascade for constraint b2_update_cascade_fkey Del /Table/114/2/"updated"/0 -CPut /Table/114/2/"updated2"/0 -> /BYTES/0x1262322d706b310001 (expecting does not exist) +CPut /Table/114/2/"updated2"/0 -> /BYTES/0x1262322d706b310001 executing cascade for constraint c1_update_cascade_fkey executing cascade for constraint c2_update_cascade_fkey executing cascade for constraint c3_id_fkey @@ -463,10 +463,10 @@ Del /Table/134/1/"original"/0 CPut /Table/134/1/"updated"/0 -> /TUPLE/ executing cascade for constraint b1_update_cascade_fkey Del /Table/135/2/"original"/0 -CPut /Table/135/2/"updated"/0 -> /BYTES/0x1262312d706b310001 (expecting does not exist) +CPut /Table/135/2/"updated"/0 -> /BYTES/0x1262312d706b310001 executing cascade for constraint b2_update_cascade_fkey Del /Table/136/2/"original"/0 -CPut /Table/136/2/"updated"/0 -> /BYTES/0x1262322d706b310001 (expecting does not exist) +CPut /Table/136/2/"updated"/0 -> /BYTES/0x1262322d706b310001 executing cascade for constraint c1_update_set_null_fkey executing cascade for constraint c2_update_set_null_fkey executing cascade for constraint c3_update_set_null_fkey @@ -751,10 +751,10 @@ Del /Table/156/1/"original"/0 CPut /Table/156/1/"updated"/0 -> /TUPLE/ executing cascade for constraint b1_update_cascade_fkey Del /Table/157/2/"original"/0 -CPut /Table/157/2/"updated"/0 -> /BYTES/0x1262312d706b310001 (expecting does not exist) +CPut /Table/157/2/"updated"/0 -> /BYTES/0x1262312d706b310001 executing cascade for constraint b2_update_cascade_fkey Del /Table/158/2/"original"/0 -CPut /Table/158/2/"updated"/0 -> /BYTES/0x1262322d706b310001 (expecting does not exist) +CPut /Table/158/2/"updated"/0 -> /BYTES/0x1262322d706b310001 executing cascade for constraint c1_update_set_null_fkey executing cascade for constraint c2_update_set_null_fkey executing cascade for constraint c3_update_set_null_fkey diff --git a/pkg/sql/opt/exec/execbuilder/testdata/partial_index b/pkg/sql/opt/exec/execbuilder/testdata/partial_index index 685ea575d905..2831023cfb0f 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/partial_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/partial_index @@ -343,8 +343,8 @@ UPDATE t SET b = 11 WHERE a = 5 Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/baz Del /Table/112/2/4/5/0 -CPut /Table/112/2/11/5/0 -> /BYTES/ (expecting does not exist) -CPut /Table/112/3/11/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/11/5/0 -> /BYTES/ +CPut /Table/112/3/11/5/0 -> /BYTES/ # Update a row that matches the first partial index before and after the update # and the index entry does not change. @@ -362,9 +362,9 @@ UPDATE t SET b = 12 WHERE a = 6 Scan /Table/112/1/6/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/6/0 -> /TUPLE/2:2:Int/12/1:3:Bytes/baz Del /Table/112/2/11/6/0 -CPut /Table/112/2/12/6/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/12/6/0 -> /BYTES/ Del /Table/112/3/11/6/0 -CPut /Table/112/3/12/6/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/3/12/6/0 -> /BYTES/ # Update a row that matches the first partial index before the update, but does # not match after the update. @@ -374,7 +374,7 @@ UPDATE t SET b = 9 WHERE a = 6 Scan /Table/112/1/6/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/6/0 -> /TUPLE/2:2:Int/9/1:3:Bytes/baz Del /Table/112/2/12/6/0 -CPut /Table/112/2/9/6/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/9/6/0 -> /BYTES/ Del /Table/112/3/12/6/0 # Update a row that matches both partial indexes before the update, the first @@ -385,9 +385,9 @@ UPDATE t SET c = 'baz', b = 12 WHERE a = 13 Scan /Table/112/1/13/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/13/0 -> /TUPLE/2:2:Int/12/1:3:Bytes/baz Del /Table/112/2/11/13/0 -CPut /Table/112/2/12/13/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/12/13/0 -> /BYTES/ Del /Table/112/3/11/13/0 -CPut /Table/112/3/12/13/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/3/12/13/0 -> /BYTES/ Del /Table/112/4/"foo"/13/0 # Reversing the previous update should reverse the partial index changes. @@ -397,10 +397,10 @@ UPDATE t SET c = 'foo', b = 11 WHERE a = 13 Scan /Table/112/1/13/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/13/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/foo Del /Table/112/2/12/13/0 -CPut /Table/112/2/11/13/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/11/13/0 -> /BYTES/ Del /Table/112/3/12/13/0 -CPut /Table/112/3/11/13/0 -> /BYTES/ (expecting does not exist) -CPut /Table/112/4/"foo"/13/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/3/11/13/0 -> /BYTES/ +CPut /Table/112/4/"foo"/13/0 -> /BYTES/ # Update a row to match a partial index that does not index the column # referenced in predicate. @@ -412,7 +412,7 @@ UPDATE u SET v = 11 WHERE k = 1 ---- Scan /Table/113/1/1/0 lock Exclusive (Block, Unreplicated) Put /Table/113/1/1/0 -> /TUPLE/2:2:Int/2/1:3:Int/11 -CPut /Table/113/2/2/1/0 -> /BYTES/ (expecting does not exist) +CPut /Table/113/2/2/1/0 -> /BYTES/ # Update a row to no longer match a partial index that does not index the column # referenced in predicate. @@ -572,7 +572,7 @@ INSERT INTO t VALUES (5, 3, 'foo') ON CONFLICT (a) DO UPDATE SET b = 3 Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/3/1:3:Bytes/bar Del /Table/112/2/4/5/0 -CPut /Table/112/2/3/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/3/5/0 -> /BYTES/ # Insert a conflicting row that does not match the first partial index # before the update, but does match after the update. @@ -582,8 +582,8 @@ INSERT INTO t VALUES (5, 7, 'foo') ON CONFLICT (a) DO UPDATE SET b = 11 Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/bar Del /Table/112/2/3/5/0 -CPut /Table/112/2/11/5/0 -> /BYTES/ (expecting does not exist) -CPut /Table/112/3/11/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/11/5/0 -> /BYTES/ +CPut /Table/112/3/11/5/0 -> /BYTES/ # Insert a conflicting row that currently matches the first partial index before # the update. Update the row so that the row no longer matches the first partial @@ -594,9 +594,9 @@ INSERT INTO t VALUES (5, 11, 'bar') ON CONFLICT (a) DO UPDATE SET b = 4, c = 'fo Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/foo Del /Table/112/2/11/5/0 -CPut /Table/112/2/4/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/4/5/0 -> /BYTES/ Del /Table/112/3/11/5/0 -CPut /Table/112/4/"foo"/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/4/"foo"/5/0 -> /BYTES/ # Insert a conflicting row that that matches the second partial index before and # after the update and the index entry does not change. @@ -606,7 +606,7 @@ INSERT INTO t VALUES (5, 11, 'bar') ON CONFLICT (a) DO UPDATE SET b = 3 Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/3/1:3:Bytes/foo Del /Table/112/2/4/5/0 -CPut /Table/112/2/3/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/3/5/0 -> /BYTES/ # Insert a conflicting row that that matches the second partial index before and # after the update and the index entry changes. @@ -616,7 +616,7 @@ INSERT INTO t VALUES (5, 11, 'bar') ON CONFLICT (a) DO UPDATE SET c = 'foobar' Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/3/1:3:Bytes/foobar Del /Table/112/4/"foo"/5/0 -CPut /Table/112/4/"foobar"/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/4/"foobar"/5/0 -> /BYTES/ # Insert a non-conflicting row that matches the first partial index. query T kvtrace @@ -660,7 +660,7 @@ INSERT INTO u VALUES (2, 3, 11) ON CONFLICT (k) DO UPDATE SET u = 4, v = 12 Scan /Table/113/1/2/0 lock Exclusive (Block, Unreplicated) Put /Table/113/1/2/0 -> /TUPLE/2:2:Int/4/1:3:Int/12 Del /Table/113/2/3/2/0 -CPut /Table/113/2/4/2/0 -> /BYTES/ (expecting does not exist) +CPut /Table/113/2/4/2/0 -> /BYTES/ # --------------------------------------------------------- # INSERT ON CONFLICT DO UPDATE primary key @@ -758,7 +758,7 @@ UPSERT INTO t VALUES (5, 3, 'bar') Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/3/1:3:Bytes/bar Del /Table/112/2/4/5/0 -CPut /Table/112/2/3/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/3/5/0 -> /BYTES/ # Upsert a conflicting row that does not match the first partial index before # the update, but does match after the update. @@ -768,8 +768,8 @@ UPSERT INTO t VALUES (5, 11, 'bar') Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/bar Del /Table/112/2/3/5/0 -CPut /Table/112/2/11/5/0 -> /BYTES/ (expecting does not exist) -CPut /Table/112/3/11/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/11/5/0 -> /BYTES/ +CPut /Table/112/3/11/5/0 -> /BYTES/ # Upsert a conflicting row that currently matches the first partial index before # the update. Update the row so that the row no longer matches the first partial @@ -780,9 +780,9 @@ UPSERT INTO t VALUES (5, 3, 'foo') Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/3/1:3:Bytes/foo Del /Table/112/2/11/5/0 -CPut /Table/112/2/3/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/3/5/0 -> /BYTES/ Del /Table/112/3/11/5/0 -CPut /Table/112/4/"foo"/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/4/"foo"/5/0 -> /BYTES/ # Upsert a conflicting row that that matches the second partial index before and # after the update and the index entry does not change. @@ -792,7 +792,7 @@ UPSERT INTO t VALUES (5, 4, 'foo') Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/foo Del /Table/112/2/3/5/0 -CPut /Table/112/2/4/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/4/5/0 -> /BYTES/ # Upsert a conflicting row that that matches the second partial index before and # after the update and the index entry changes. @@ -802,7 +802,7 @@ UPSERT INTO t VALUES (5, 4, 'foobar') Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/112/1/5/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/foobar Del /Table/112/4/"foo"/5/0 -CPut /Table/112/4/"foobar"/5/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/4/"foobar"/5/0 -> /BYTES/ # Upsert a non-conflicting row that matches the first partial index. query T kvtrace @@ -846,7 +846,7 @@ UPSERT INTO u VALUES (2, 4, 12) Scan /Table/113/1/2/0 lock Exclusive (Block, Unreplicated) Put /Table/113/1/2/0 -> /TUPLE/2:2:Int/4/1:3:Int/12 Del /Table/113/2/3/2/0 -CPut /Table/113/2/4/2/0 -> /BYTES/ (expecting does not exist) +CPut /Table/113/2/4/2/0 -> /BYTES/ # Tests for partial inverted indexes. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families b/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families index 1844cc6bdaad..4ee6bea32c5a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families +++ b/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families @@ -406,7 +406,7 @@ UPDATE t SET b = 4, c = NULL, d = NULL, e = 7, f = NULL WHERE y = 2 ---- Put /Table/112/2/2/1/2/1 -> /TUPLE/3:3:Int/3/1:4:Int/4 Del /Table/112/2/2/1/3/1 -CPut /Table/112/2/2/1/4/1 -> /TUPLE/7:7:Int/7 (expecting does not exist) +CPut /Table/112/2/2/1/4/1 -> /TUPLE/7:7:Int/7 Del /Table/112/2/2/1/5/1 query IIIIIIII @@ -422,10 +422,10 @@ INSERT INTO t VALUES (3, 3, NULL, NULL, NULL, NULL, NULL, NULL) query T kvtrace(Put,Del,CPut,prefix=/Table/112/2/) UPDATE t SET a = 10, b = 11, c = 12, d = 13, e = 14, f = 15 WHERE y = 3 ---- -CPut /Table/112/2/3/3/2/1 -> /TUPLE/3:3:Int/10/1:4:Int/11 (expecting does not exist) -CPut /Table/112/2/3/3/3/1 -> /TUPLE/5:5:Int/12/1:6:Int/13 (expecting does not exist) -CPut /Table/112/2/3/3/4/1 -> /TUPLE/7:7:Int/14 (expecting does not exist) -CPut /Table/112/2/3/3/5/1 -> /TUPLE/8:8:Int/15 (expecting does not exist) +CPut /Table/112/2/3/3/2/1 -> /TUPLE/3:3:Int/10/1:4:Int/11 +CPut /Table/112/2/3/3/3/1 -> /TUPLE/5:5:Int/12/1:6:Int/13 +CPut /Table/112/2/3/3/4/1 -> /TUPLE/7:7:Int/14 +CPut /Table/112/2/3/3/5/1 -> /TUPLE/8:8:Int/15 # Test a case where the update causes all k/v's other than # the sentinel k/v to get deleted. @@ -447,13 +447,13 @@ query T kvtrace(Put,Del,CPut,prefix=/Table/112/2/) UPDATE t SET y = 22 WHERE y = 21 ---- Del /Table/112/2/21/20/0 -CPut /Table/112/2/22/20/0 -> /BYTES/ (expecting does not exist) +CPut /Table/112/2/22/20/0 -> /BYTES/ Del /Table/112/2/21/20/2/1 -CPut /Table/112/2/22/20/2/1 -> /TUPLE/3:3:Int/22 (expecting does not exist) +CPut /Table/112/2/22/20/2/1 -> /TUPLE/3:3:Int/22 Del /Table/112/2/21/20/3/1 -CPut /Table/112/2/22/20/3/1 -> /TUPLE/6:6:Int/25 (expecting does not exist) +CPut /Table/112/2/22/20/3/1 -> /TUPLE/6:6:Int/25 Del /Table/112/2/21/20/5/1 -CPut /Table/112/2/22/20/5/1 -> /TUPLE/8:8:Int/27 (expecting does not exist) +CPut /Table/112/2/22/20/5/1 -> /TUPLE/8:8:Int/27 # Ensure that the final results on both indexes make sense. query IIIIIIII rowsort @@ -489,7 +489,7 @@ query T kvtrace(Put,CPut,Del,prefix=/Table/113/2/) UPDATE t SET y = 5 where y = 2 ---- Del /Table/113/2/2/1/0 -CPut /Table/113/2/5/1/0 -> /BYTES/0x33061308 (expecting does not exist) +CPut /Table/113/2/5/1/0 -> /BYTES/0x33061308 # Changing the value just results in a cput. query T kvtrace(Put,Del,CPut,prefix=/Table/113/2/) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index 32d970efe426..d30539b525c2 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -1099,7 +1099,7 @@ colbatchscan Scan /Table/120/1/2/0 lock Exclusive (Block, Unreplicated) colbatchscan fetched: /kv/kv_pkey/2/v -> /3 count Put /Table/120/1/2/0 -> /TUPLE/2:2:Int/2 count Del /Table/120/2/3/0 -count CPut /Table/120/2/2/0 -> /BYTES/0x8a (expecting does not exist) +count CPut /Table/120/2/2/0 -> /BYTES/0x8a sql query execution failed after 0 rows: duplicate key value violates unique constraint "woo" # Test that implicit SELECT FOR UPDATE doesn't spread to subqueries. diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index 29c2c0770639..7eed190609a1 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -431,13 +431,13 @@ func (ru *Updater) UpdateRow( if sameKey { log.VEventf(ctx, 2, "Put %s -> %v", k, v) } else { - log.VEventf(ctx, 2, "CPut %s -> %v (expecting does not exist)", k, v) + log.VEventf(ctx, 2, "CPut %s -> %v", k, v) } } if sameKey { batch.Put(newEntry.Key, &newEntry.Value) } else { - batch.CPutAllowingIfNotExists(newEntry.Key, &newEntry.Value, nil /* expValue */) + batch.CPut(newEntry.Key, &newEntry.Value, nil /* expValue */) } } writtenIndexes.Add(i) @@ -472,7 +472,7 @@ func (ru *Updater) UpdateRow( if traceKV { k := keys.PrettyPrint(ru.Helper.secIndexValDirs[i], newEntry.Key) v := newEntry.Value.PrettyPrint() - log.VEventf(ctx, 2, "CPut %s -> %v (expecting does not exist)", k, v) + log.VEventf(ctx, 2, "CPut %s -> %v", k, v) } batch.CPut(newEntry.Key, &newEntry.Value, nil) } @@ -506,7 +506,7 @@ func (ru *Updater) UpdateRow( if traceKV { k := keys.PrettyPrint(ru.Helper.secIndexValDirs[i], newEntry.Key) v := newEntry.Value.PrettyPrint() - log.VEventf(ctx, 2, "CPut %s -> %v (expecting does not exist)", k, v) + log.VEventf(ctx, 2, "CPut %s -> %v", k, v) } batch.CPut(newEntry.Key, &newEntry.Value, nil) } diff --git a/pkg/sql/testdata/index_mutations/merging b/pkg/sql/testdata/index_mutations/merging index fecbdff32eb8..ae2b8ef81f09 100644 --- a/pkg/sql/testdata/index_mutations/merging +++ b/pkg/sql/testdata/index_mutations/merging @@ -148,7 +148,7 @@ UPDATE tefp SET a = a + 1, b = b + 100 Scan /Table/108/{1-2} lock Exclusive (Block, Unreplicated) Put /Table/108/1/1/0 -> /TUPLE/2:2:Int/3/1:3:Int/200 Del /Table/108/2/102/0 -CPut /Table/108/2/203/0 -> /BYTES/0x89 (expecting does not exist) +CPut /Table/108/2/203/0 -> /BYTES/0x89 # Update a row with different values without changing the index entry. kvtrace From 6ea502f52e011bb0dbf962de84773c78be94b3a1 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Tue, 14 Jan 2025 16:11:23 -0500 Subject: [PATCH 107/126] spanconfigkvsubscriberccl: run expensive tests in heavy pool We have seen this timeout under race over the past few days, so this patch gives the test more resources under race/deadlock. Release note: None --- pkg/ccl/spanconfigccl/spanconfigkvsubscriberccl/BUILD.bazel | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/ccl/spanconfigccl/spanconfigkvsubscriberccl/BUILD.bazel b/pkg/ccl/spanconfigccl/spanconfigkvsubscriberccl/BUILD.bazel index c059b0eb495a..64c493c1d288 100644 --- a/pkg/ccl/spanconfigccl/spanconfigkvsubscriberccl/BUILD.bazel +++ b/pkg/ccl/spanconfigccl/spanconfigkvsubscriberccl/BUILD.bazel @@ -7,7 +7,7 @@ go_test( "main_test.go", ], exec_properties = select({ - "//build/toolchains:is_heavy": {"test.Pool": "large"}, + "//build/toolchains:is_heavy": {"test.Pool": "heavy"}, "//conditions:default": {"test.Pool": "default"}, }), deps = [ From c79ea8a2bc08d031dc2916be6ce21f3637fe6f77 Mon Sep 17 00:00:00 2001 From: Manu Gomez Date: Tue, 14 Jan 2025 16:44:00 -0500 Subject: [PATCH 108/126] authors: add Manu Gomez to authors Epic: None Release note: None --- AUTHORS | 1 + 1 file changed, 1 insertion(+) diff --git a/AUTHORS b/AUTHORS index 6f6adc6d5d38..88eeb50e83c4 100644 --- a/AUTHORS +++ b/AUTHORS @@ -322,6 +322,7 @@ Madhav Suresh madhavsuresh Mahmoud Al-Qudsi Maitri Morarji Manik Surtani +Manu Gomez InManuBytes Marc Berhault marc MBerhault Marc Shirley Marcus Gartner From bf608152e604253b72a65480a3ad1628065166d5 Mon Sep 17 00:00:00 2001 From: Andy Yang Date: Mon, 13 Jan 2025 16:16:17 -0500 Subject: [PATCH 109/126] changefeedccl: rename changefeed.frontier_checkpoint_frequency setting This patch renames `changefeed.frontier_checkpoint_frequency` to `changefeed.span_checkpoint.interval` for consistency. Release note: None --- pkg/ccl/changefeedccl/alter_changefeed_test.go | 6 +++--- pkg/ccl/changefeedccl/changefeed_processors.go | 10 +++++----- pkg/ccl/changefeedccl/changefeed_test.go | 10 +++++----- pkg/ccl/changefeedccl/changefeedbase/settings.go | 11 +++++++---- pkg/cmd/roachtest/tests/cdc.go | 4 ++-- pkg/cmd/roachtest/tests/cdc_bench.go | 2 +- 6 files changed, 23 insertions(+), 20 deletions(-) diff --git a/pkg/ccl/changefeedccl/alter_changefeed_test.go b/pkg/ccl/changefeedccl/alter_changefeed_test.go index bd93464d5166..764d0b94d568 100644 --- a/pkg/ccl/changefeedccl/alter_changefeed_test.go +++ b/pkg/ccl/changefeedccl/alter_changefeed_test.go @@ -1219,9 +1219,9 @@ func TestAlterChangefeedAddTargetsDuringSchemaChangeError(t *testing.T) { var maxCheckpointSize int64 = 100 << 20 // Ensure that checkpoints happen every time by setting a large checkpoint size. - // Because setting 0 for the FrontierCheckpointFrequency disables checkpointing, + // Because setting 0 for the SpanCheckpointInterval disables checkpointing, // setting 1 nanosecond is the smallest possible value. - changefeedbase.FrontierCheckpointFrequency.Override( + changefeedbase.SpanCheckpointInterval.Override( context.Background(), &s.Server.ClusterSettings().SV, 1*time.Nanosecond) changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, maxCheckpointSize) @@ -1385,7 +1385,7 @@ func TestAlterChangefeedAddTargetsDuringBackfill(t *testing.T) { } // Checkpoint progress frequently, and set the checkpoint size limit. - changefeedbase.FrontierCheckpointFrequency.Override( + changefeedbase.SpanCheckpointInterval.Override( context.Background(), &s.Server.ClusterSettings().SV, 1) changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, maxCheckpointSize) diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index c0848b046b92..94630f8fb3ce 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -1094,11 +1094,11 @@ func newJobState( } func canCheckpointSpans(sv *settings.Values, lastCheckpoint time.Time) bool { - freq := changefeedbase.FrontierCheckpointFrequency.Get(sv) - if freq == 0 { + interval := changefeedbase.SpanCheckpointInterval.Get(sv) + if interval == 0 { return false } - return timeutil.Since(lastCheckpoint) > freq + return timeutil.Since(lastCheckpoint) > interval } func (j *jobState) canCheckpointSpans() bool { @@ -1311,9 +1311,9 @@ func (cf *changeFrontier) Start(ctx context.Context) { return } cf.js.job = job - if changefeedbase.FrontierCheckpointFrequency.Get(&cf.FlowCtx.Cfg.Settings.SV) == 0 { + if changefeedbase.SpanCheckpointInterval.Get(&cf.FlowCtx.Cfg.Settings.SV) == 0 { log.Warning(ctx, - "Frontier checkpointing disabled; set changefeed.frontier_checkpoint_frequency to non-zero value to re-enable") + "span-level checkpointing disabled; set changefeed.span_checkpoint.interval to positive duration to re-enable") } // Recover highwater information from job progress. diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 9968b5798890..94222df075aa 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -2336,7 +2336,7 @@ func TestChangefeedLaggingSpanCheckpointing(t *testing.T) { // Checkpoint progress frequently, allow a large enough checkpoint, and // reduce the lag threshold to allow lag checkpointing to trigger - changefeedbase.FrontierCheckpointFrequency.Override( + changefeedbase.SpanCheckpointInterval.Override( context.Background(), &s.ClusterSettings().SV, 10*time.Millisecond) changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.ClusterSettings().SV, 100<<20) @@ -2522,7 +2522,7 @@ func TestChangefeedSchemaChangeBackfillCheckpoint(t *testing.T) { require.NoError(t, jobFeed.Pause()) // Checkpoint progress frequently, and set the checkpoint size limit. - changefeedbase.FrontierCheckpointFrequency.Override( + changefeedbase.SpanCheckpointInterval.Override( context.Background(), &s.Server.ClusterSettings().SV, 1) changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, maxCheckpointSize) @@ -7341,7 +7341,7 @@ func TestChangefeedBackfillCheckpoint(t *testing.T) { } // Checkpoint progress frequently, and set the checkpoint size limit. - changefeedbase.FrontierCheckpointFrequency.Override( + changefeedbase.SpanCheckpointInterval.Override( context.Background(), &s.Server.ClusterSettings().SV, 1) changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, maxCheckpointSize) @@ -7492,7 +7492,7 @@ func TestCoreChangefeedBackfillScanCheckpoint(t *testing.T) { b.Header.MaxSpanRequestKeys = 1 + rnd.Int63n(25) return nil } - changefeedbase.FrontierCheckpointFrequency.Override( + changefeedbase.SpanCheckpointInterval.Override( context.Background(), &s.Server.ClusterSettings().SV, 1) changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.Server.ClusterSettings().SV, 100<<20) @@ -9936,7 +9936,7 @@ func TestChangefeedProtectedTimestampUpdate(t *testing.T) { // Checkpoint and trigger potential protected timestamp updates frequently. // Make the protected timestamp lag long enough that it shouldn't be // immediately updated after a restart. - changefeedbase.FrontierCheckpointFrequency.Override( + changefeedbase.SpanCheckpointInterval.Override( context.Background(), &s.Server.ClusterSettings().SV, 10*time.Millisecond) changefeedbase.ProtectTimestampInterval.Override( context.Background(), &s.Server.ClusterSettings().SV, 10*time.Millisecond) diff --git a/pkg/ccl/changefeedccl/changefeedbase/settings.go b/pkg/ccl/changefeedccl/changefeedbase/settings.go index c1ae7aec921e..f28590b92ed0 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/settings.go +++ b/pkg/ccl/changefeedccl/changefeedbase/settings.go @@ -67,13 +67,16 @@ var IdleTimeout = settings.RegisterDurationSetting( settings.WithName("changefeed.auto_idle.timeout"), ) -// FrontierCheckpointFrequency controls the frequency of frontier checkpoints. -var FrontierCheckpointFrequency = settings.RegisterDurationSetting( +// SpanCheckpointInterval controls how often span-level checkpoints +// can be written. +var SpanCheckpointInterval = settings.RegisterDurationSetting( settings.ApplicationLevel, "changefeed.frontier_checkpoint_frequency", - "controls the frequency with which span level checkpoints will be written; if 0, disabled", + "interval at which span-level checkpoints will be written; "+ + "if 0, span-level checkpoints are disabled", 10*time.Minute, settings.NonNegativeDuration, + settings.WithName("changefeed.span_checkpoint.interval"), ) // FrontierHighwaterLagCheckpointThreshold controls the amount the high-water @@ -100,7 +103,7 @@ var FrontierHighwaterLagCheckpointThreshold = settings.RegisterDurationSetting( // - Assume we want to have at most 150MB worth of checkpoints in the job record. // // Therefore, we should write at most 6 MB of checkpoint/hour; OR, based on the default -// FrontierCheckpointFrequency setting, 1 MB per checkpoint. +// SpanCheckpointInterval setting, 1 MB per checkpoint. var SpanCheckpointMaxBytes = settings.RegisterByteSizeSetting( settings.ApplicationLevel, "changefeed.frontier_checkpoint_max_bytes", diff --git a/pkg/cmd/roachtest/tests/cdc.go b/pkg/cmd/roachtest/tests/cdc.go index 3cdde1b96141..44a1a90490e5 100644 --- a/pkg/cmd/roachtest/tests/cdc.go +++ b/pkg/cmd/roachtest/tests/cdc.go @@ -1047,13 +1047,13 @@ func runCDCInitialScanRollingRestart( switch checkpointType { case cdcNormalCheckpoint: setupStmts = append(setupStmts, - `SET CLUSTER SETTING changefeed.frontier_checkpoint_frequency = '1s'`, + `SET CLUSTER SETTING changefeed.span_checkpoint.interval = '1s'`, `SET CLUSTER SETTING changefeed.shutdown_checkpoint.enabled = 'false'`, ) case cdcShutdownCheckpoint: const largeSplitCount = 5 setupStmts = append(setupStmts, - `SET CLUSTER SETTING changefeed.frontier_checkpoint_frequency = '0'`, + `SET CLUSTER SETTING changefeed.span_checkpoint.interval = '0'`, `SET CLUSTER SETTING changefeed.shutdown_checkpoint.enabled = 'true'`, // Split some bigger chunks up to scatter it a bit more. fmt.Sprintf(`ALTER TABLE large SPLIT AT SELECT id FROM large ORDER BY random() LIMIT %d`, largeSplitCount/4), diff --git a/pkg/cmd/roachtest/tests/cdc_bench.go b/pkg/cmd/roachtest/tests/cdc_bench.go index fd17c9ea067f..d501c48835d3 100644 --- a/pkg/cmd/roachtest/tests/cdc_bench.go +++ b/pkg/cmd/roachtest/tests/cdc_bench.go @@ -173,7 +173,7 @@ func makeCDCBenchOptions(c cluster.Cluster) (option.StartOpts, install.ClusterSe // Checkpoint frequently. Some of the larger benchmarks might overload the // cluster. Producing frequent span-level checkpoints helps with recovery. - settings.ClusterSettings["changefeed.frontier_checkpoint_frequency"] = "60s" + settings.ClusterSettings["changefeed.span_checkpoint.interval"] = "60s" settings.ClusterSettings["changefeed.frontier_highwater_lag_checkpoint_threshold"] = "30s" // Bump up the number of allowed catchup scans. Doing catchup for 100k ranges with default From 64f47b4db69f374818677177278a2e4dd8c5499b Mon Sep 17 00:00:00 2001 From: Andy Yang Date: Tue, 14 Jan 2025 15:10:43 -0500 Subject: [PATCH 110/126] changefeedccl: rename changefeed.frontier_highwater_lag_checkpoint_threshold setting This patch renames `changefeed.frontier_highwater_lag_checkpoint_threshold` to `changefeed.span_checkpoint.lag_threshold` for consistency. Release note (ops change): The cluster setting `changefeed.frontier_highwater_lag_checkpoint_threshold` has been renamed to `changefeed.span_checkpoint.lag_threshold`. The old name remains available for backwards-compatibility. --- .../generated/settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/ccl/changefeedccl/changefeed_test.go | 2 +- .../changefeedccl/changefeedbase/settings.go | 17 +++++++++++------ pkg/ccl/changefeedccl/resolvedspan/frontier.go | 4 ++-- pkg/cmd/roachtest/tests/cdc_bench.go | 2 +- 6 files changed, 17 insertions(+), 12 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index b319585a02da..5731bb6fa7d9 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -17,7 +17,7 @@ changefeed.default_range_distribution_strategy enumeration default configures ho changefeed.event_consumer_worker_queue_size integer 16 if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of events which a worker can buffer application changefeed.event_consumer_workers integer 0 the number of workers to use when processing events: <0 disables, 0 assigns a reasonable default, >0 assigns the setting value. for experimental/core changefeeds and changefeeds using parquet format, this is disabled application changefeed.fast_gzip.enabled boolean true use fast gzip implementation application -changefeed.frontier_highwater_lag_checkpoint_threshold duration 10m0s controls the maximum the high-water mark is allowed to lag behind the leading spans of the frontier before per-span checkpointing is enabled; if 0, checkpointing due to high-water lag is disabled application +changefeed.span_checkpoint.lag_threshold (alias: changefeed.frontier_highwater_lag_checkpoint_threshold) duration 10m0s the amount of time a changefeed's lagging (slowest) spans must lag behind its leading (fastest) spans before a span-level checkpoint to save leading span progress is written; if 0, span-level checkpoints due to lagging spans is disabled application changefeed.memory.per_changefeed_limit byte size 512 MiB controls amount of data that can be buffered per changefeed application changefeed.resolved_timestamp.min_update_interval (alias: changefeed.min_highwater_advance) duration 0s minimum amount of time that must have elapsed since the last time a changefeed's resolved timestamp was updated before it is eligible to be updated again; default of 0 means no minimum interval is enforced but updating will still be limited by the average time it takes to checkpoint progress application changefeed.node_throttle_config string specifies node level throttling configuration for all changefeeeds application diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 4c11feeb0ea4..494baac071e0 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -22,7 +22,7 @@ - + diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 94222df075aa..3e7fbf2c1bc8 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -2340,7 +2340,7 @@ func TestChangefeedLaggingSpanCheckpointing(t *testing.T) { context.Background(), &s.ClusterSettings().SV, 10*time.Millisecond) changefeedbase.SpanCheckpointMaxBytes.Override( context.Background(), &s.ClusterSettings().SV, 100<<20) - changefeedbase.FrontierHighwaterLagCheckpointThreshold.Override( + changefeedbase.SpanCheckpointLagThreshold.Override( context.Background(), &s.ClusterSettings().SV, 10*time.Millisecond) // We'll start changefeed with the cursor. diff --git a/pkg/ccl/changefeedccl/changefeedbase/settings.go b/pkg/ccl/changefeedccl/changefeedbase/settings.go index f28590b92ed0..99673a19ae2c 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/settings.go +++ b/pkg/ccl/changefeedccl/changefeedbase/settings.go @@ -79,16 +79,21 @@ var SpanCheckpointInterval = settings.RegisterDurationSetting( settings.WithName("changefeed.span_checkpoint.interval"), ) -// FrontierHighwaterLagCheckpointThreshold controls the amount the high-water -// mark is allowed to lag behind the leading edge of the frontier before we -// begin to attempt checkpointing spans above the high-water mark -var FrontierHighwaterLagCheckpointThreshold = settings.RegisterDurationSetting( +// SpanCheckpointLagThreshold controls the amount of time a changefeed's +// lagging spans must lag behind its leading spans before a span-level +// checkpoint is written. +var SpanCheckpointLagThreshold = settings.RegisterDurationSetting( settings.ApplicationLevel, "changefeed.frontier_highwater_lag_checkpoint_threshold", - "controls the maximum the high-water mark is allowed to lag behind the leading spans of the frontier before per-span checkpointing is enabled; if 0, checkpointing due to high-water lag is disabled", + "the amount of time a changefeed's lagging (slowest) spans must lag "+ + "behind its leading (fastest) spans before a span-level checkpoint "+ + "to save leading span progress is written; if 0, span-level checkpoints "+ + "due to lagging spans is disabled", 10*time.Minute, settings.NonNegativeDuration, - settings.WithPublic) + settings.WithPublic, + settings.WithName("changefeed.span_checkpoint.lag_threshold"), +) // SpanCheckpointMaxBytes controls the maximum number of key bytes that will be added // to a span-level checkpoint record. diff --git a/pkg/ccl/changefeedccl/resolvedspan/frontier.go b/pkg/ccl/changefeedccl/resolvedspan/frontier.go index 93e348e04258..ad8d017e212c 100644 --- a/pkg/ccl/changefeedccl/resolvedspan/frontier.go +++ b/pkg/ccl/changefeedccl/resolvedspan/frontier.go @@ -295,9 +295,9 @@ func (f *resolvedSpanFrontier) assertBoundaryNotEarlier( // HasLaggingSpans returns whether the frontier has lagging spans as defined // by whether the frontier trails the latest timestamp by at least -// changefeedbase.FrontierHighwaterLagCheckpointThreshold. +// changefeedbase.SpanCheckpointLagThreshold. func (f *resolvedSpanFrontier) HasLaggingSpans(sv *settings.Values) bool { - lagThresholdNanos := int64(changefeedbase.FrontierHighwaterLagCheckpointThreshold.Get(sv)) + lagThresholdNanos := int64(changefeedbase.SpanCheckpointLagThreshold.Get(sv)) if lagThresholdNanos == 0 { return false } diff --git a/pkg/cmd/roachtest/tests/cdc_bench.go b/pkg/cmd/roachtest/tests/cdc_bench.go index d501c48835d3..2eec7fce6b1b 100644 --- a/pkg/cmd/roachtest/tests/cdc_bench.go +++ b/pkg/cmd/roachtest/tests/cdc_bench.go @@ -174,7 +174,7 @@ func makeCDCBenchOptions(c cluster.Cluster) (option.StartOpts, install.ClusterSe // Checkpoint frequently. Some of the larger benchmarks might overload the // cluster. Producing frequent span-level checkpoints helps with recovery. settings.ClusterSettings["changefeed.span_checkpoint.interval"] = "60s" - settings.ClusterSettings["changefeed.frontier_highwater_lag_checkpoint_threshold"] = "30s" + settings.ClusterSettings["changefeed.span_checkpoint.lag_threshold"] = "30s" // Bump up the number of allowed catchup scans. Doing catchup for 100k ranges with default // configuration (8 client side, 16 per store) takes a while (~1500-2000 ranges per min minutes). From d40fe61fd7d1d4629ae0cd26d9dcc672c3f9124b Mon Sep 17 00:00:00 2001 From: CRL Release bot Date: Wed, 15 Jan 2025 00:21:49 +0000 Subject: [PATCH 111/126] master: Update pkg/testutils/release/cockroach_releases.yaml Update pkg/testutils/release/cockroach_releases.yaml with recent values. Epic: None Release note: None Release justification: test-only updates --- pkg/sql/logictest/REPOSITORIES.bzl | 20 +++++++++---------- pkg/testutils/release/cockroach_releases.yaml | 8 ++++---- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/pkg/sql/logictest/REPOSITORIES.bzl b/pkg/sql/logictest/REPOSITORIES.bzl index 2fbb7da43d6e..27b5f6774586 100644 --- a/pkg/sql/logictest/REPOSITORIES.bzl +++ b/pkg/sql/logictest/REPOSITORIES.bzl @@ -7,17 +7,17 @@ CONFIG_DARWIN_AMD64 = "darwin-10.9-amd64" CONFIG_DARWIN_ARM64 = "darwin-11.0-arm64" _CONFIGS = [ - ("24.2.7", [ - (CONFIG_DARWIN_AMD64, "e2e411277fa62b5c84213f744bcfcfb1d1ff01b1bba9739da7f5fa69821e851b"), - (CONFIG_DARWIN_ARM64, "23b50510c0272267b77a9fcff684f5a9d492b5d2de22058fea601cd9a1cce75b"), - (CONFIG_LINUX_AMD64, "09ca3a221e7fc4842ae5830759dd944173d93b376707357ebb5e31d580e73390"), - (CONFIG_LINUX_ARM64, "be7942c88f5e55c9272a1f9961c685737091897c305247766668f8405ceae59a"), + ("24.2.8", [ + (CONFIG_DARWIN_AMD64, "616e8fe3f5cf9d98351aeedc2ac2b46cc3851f87d56387d5e25323fea7788932"), + (CONFIG_DARWIN_ARM64, "90c06f4f957264f51cc2011c181fa37cd752cb6dc150ff41eeb42b6ea12c5f40"), + (CONFIG_LINUX_AMD64, "96e759169c7c404cfdbdcdca1e1a99b493627cb63b00c98d364ccf3190986a27"), + (CONFIG_LINUX_ARM64, "6c736559ccf0ca77aca4e67448515f648ddfdf089c0e14be48266e6cb435ce04"), ]), - ("24.3.2", [ - (CONFIG_DARWIN_AMD64, "9b500e5e99f6f8460d0b49a7ac2094c7ae699345997a498abd985e348ae5e72c"), - (CONFIG_DARWIN_ARM64, "07d318d23440579c2531af305c50c62e51edb5e28c63f67d4f7f3c6829d4d801"), - (CONFIG_LINUX_AMD64, "9eead2cf7d8619b0d8cf40a11cb960b5f268b5e71a6265e1a1fc6825f7caa17b"), - (CONFIG_LINUX_ARM64, "7cdba86e2937ae7fa5be889112cd1045d61d5c83f1921e0b7cbe5e2b50da5328"), + ("24.3.3", [ + (CONFIG_DARWIN_AMD64, "da65eff3d213eb71e40dcd81cc63d1fc48ee384054d85201c4a5111e141f25ba"), + (CONFIG_DARWIN_ARM64, "028a25de90578811490e667cee20b77a008e6fd847130ee1a7baee9845d52996"), + (CONFIG_LINUX_AMD64, "ca6beecec742e62a9c01a3aa99e19a567af102c779462ecb971d1d5bc4dd9edc"), + (CONFIG_LINUX_ARM64, "667e906e42ef7991f35cb79004fdd37ff398d3a54d30584afc6eea73f19a5d13"), ]), ] diff --git a/pkg/testutils/release/cockroach_releases.yaml b/pkg/testutils/release/cockroach_releases.yaml index ab4f95de9184..789f33bcf31a 100644 --- a/pkg/testutils/release/cockroach_releases.yaml +++ b/pkg/testutils/release/cockroach_releases.yaml @@ -19,18 +19,18 @@ - 23.1.0 predecessor: "22.2" "23.2": - latest: 23.2.18 + latest: 23.2.19 predecessor: "23.1" "24.1": - latest: 24.1.9 + latest: 24.1.10 predecessor: "23.2" "24.2": - latest: 24.2.7 + latest: 24.2.8 withdrawn: - 24.2.1 predecessor: "24.1" "24.3": - latest: 24.3.2 + latest: 24.3.3 predecessor: "24.2" "25.1": predecessor: "24.3" From fab0181275ad2d0428064dfe8df22367a7738723 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Tue, 14 Jan 2025 13:21:38 -0500 Subject: [PATCH 112/126] storage/disk: don't error in absence of collected states Previously, the absence of disk stats was considered an error by the disk monitor. In non-linux platforms, we don't have disk stats. This error was spamming the cockroach logs. Epic: none Release note: none --- pkg/storage/disk/monitor.go | 4 +--- pkg/storage/disk/monitor_tracer.go | 11 +++++------ pkg/storage/disk/monitor_tracer_test.go | 8 ++------ pkg/storage/disk/testdata/tracer | 2 +- 4 files changed, 9 insertions(+), 16 deletions(-) diff --git a/pkg/storage/disk/monitor.go b/pkg/storage/disk/monitor.go index 518b3dbe4e8e..8bfab117caaa 100644 --- a/pkg/storage/disk/monitor.go +++ b/pkg/storage/disk/monitor.go @@ -229,9 +229,7 @@ type Monitor struct { // CumulativeStats returns the most-recent stats observed. func (m *Monitor) CumulativeStats() (Stats, error) { - if event, err := m.tracer.Latest(); err != nil { - return Stats{}, err - } else if event.err != nil { + if event := m.tracer.Latest(); event.err != nil { return Stats{}, event.err } else { return event.stats, nil diff --git a/pkg/storage/disk/monitor_tracer.go b/pkg/storage/disk/monitor_tracer.go index b244a2f797ea..b0839a20f5f7 100644 --- a/pkg/storage/disk/monitor_tracer.go +++ b/pkg/storage/disk/monitor_tracer.go @@ -13,7 +13,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/errors" ) type traceEvent struct { @@ -87,16 +86,16 @@ func (m *monitorTracer) RecordEvent(event traceEvent) { } } -// Latest retrieves the last traceEvent that was queued. If the trace is empty -// we throw an error. -func (m *monitorTracer) Latest() (traceEvent, error) { +// Latest retrieves the last traceEvent that was queued. Returns a zero-valued +// traceEvent if none exists. +func (m *monitorTracer) Latest() traceEvent { m.mu.Lock() defer m.mu.Unlock() if m.sizeLocked() == 0 { - return traceEvent{}, errors.Errorf("trace is empty") + return traceEvent{} } latestIdx := (m.mu.end - 1) % m.capacity - return m.mu.trace[latestIdx], nil + return m.mu.trace[latestIdx] } // RollingWindow retrieves all traceEvents that occurred after the specified diff --git a/pkg/storage/disk/monitor_tracer_test.go b/pkg/storage/disk/monitor_tracer_test.go index 573266c1f40c..9cbdd4da2a6a 100644 --- a/pkg/storage/disk/monitor_tracer_test.go +++ b/pkg/storage/disk/monitor_tracer_test.go @@ -90,13 +90,9 @@ func TestMonitorTracer(t *testing.T) { } return "" case "latest": - event, err := tracer.Latest() + event := tracer.Latest() buf.Reset() - if err != nil { - fmt.Fprint(&buf, err.Error()) - } else { - fmt.Fprintf(&buf, "%q", event) - } + fmt.Fprintf(&buf, "%q", event) return buf.String() case "trace": buf.Reset() diff --git a/pkg/storage/disk/testdata/tracer b/pkg/storage/disk/testdata/tracer index 4d05cb749ac7..941ff721e203 100644 --- a/pkg/storage/disk/testdata/tracer +++ b/pkg/storage/disk/testdata/tracer @@ -8,7 +8,7 @@ rolling-window time=2024-03-27T12:00:00.5Z latest ---- -trace is empty +"0001-01-01T00:00:00Z\t\t0\t0\t0\t0s\t0\t0\t0\t0s\t0\t0s\t0s\t0\t0\t0\t0s\t0\t0s\tnil" trace ---- From 4c8ee35d083948fc34a43c4dd8a9c756e73b5c9f Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Mon, 13 Jan 2025 15:58:43 -0500 Subject: [PATCH 113/126] go.mod: bump Pebble to 3748221737d4 Changes: * [`37482217`](https://github.com/cockroachdb/pebble/commit/37482217) sstable/block: add Reader type * [`8d3363b3`](https://github.com/cockroachdb/pebble/commit/8d3363b3) sstable/block: move category stats, read env * [`a18792eb`](https://github.com/cockroachdb/pebble/commit/a18792eb) sstable: use errorfs in TestIteratorErrorOnInit Release note: none. Epic: none. --- DEPS.bzl | 6 ++-- build/bazelutil/distdir_files.bzl | 2 +- go.mod | 2 +- go.sum | 4 +-- .../backup-restore/restore-validation-only | 2 +- pkg/kv/kvserver/BUILD.bazel | 2 +- pkg/kv/kvserver/metrics.go | 12 ++++---- pkg/storage/fs/BUILD.bazel | 2 +- pkg/storage/fs/category.go | 28 +++++++++---------- pkg/storage/pebble.go | 6 ++-- pkg/storage/pebble_test.go | 4 +-- 11 files changed, 35 insertions(+), 35 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 0d1970d9b3e6..2d439b1df337 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -1885,10 +1885,10 @@ def go_deps(): patches = [ "@com_github_cockroachdb_cockroach//build/patches:com_github_cockroachdb_pebble.patch", ], - sha256 = "6a36c6ed7f11452d9866fd3e95430a02020a8ed478dac9cde3e60b173904aeb8", - strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20250111211125-38fb0512c50a", + sha256 = "723fdfb0271c1d50ac9ebb710a9eeb8f249867f918056c0b166c30d13290dc95", + strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20250113205511-3748221737d4", urls = [ - "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20250111211125-38fb0512c50a.zip", + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20250113205511-3748221737d4.zip", ], ) go_repository( diff --git a/build/bazelutil/distdir_files.bzl b/build/bazelutil/distdir_files.bzl index ba24dc4d7d19..950c49f1691a 100644 --- a/build/bazelutil/distdir_files.bzl +++ b/build/bazelutil/distdir_files.bzl @@ -358,7 +358,7 @@ DISTDIR_FILES = { "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/gostdlib/com_github_cockroachdb_gostdlib-v1.19.0.zip": "c4d516bcfe8c07b6fc09b8a9a07a95065b36c2855627cb3514e40c98f872b69e", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/logtags/com_github_cockroachdb_logtags-v0.0.0-20241215232642-bb51bb14a506.zip": "920068af09e3846d9ebb4e4a7787ff1dd10f3989c5f940ad861b0f6a9f824f6e", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/metamorphic/com_github_cockroachdb_metamorphic-v0.0.0-20231108215700-4ba948b56895.zip": "28c8cf42192951b69378cf537be5a9a43f2aeb35542908cc4fe5f689505853ea", - "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20250111211125-38fb0512c50a.zip": "6a36c6ed7f11452d9866fd3e95430a02020a8ed478dac9cde3e60b173904aeb8", + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20250113205511-3748221737d4.zip": "723fdfb0271c1d50ac9ebb710a9eeb8f249867f918056c0b166c30d13290dc95", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/redact/com_github_cockroachdb_redact-v1.1.5.zip": "11b30528eb0dafc8bc1a5ba39d81277c257cbe6946a7564402f588357c164560", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/returncheck/com_github_cockroachdb_returncheck-v0.0.0-20200612231554-92cdbca611dd.zip": "ce92ba4352deec995b1f2eecf16eba7f5d51f5aa245a1c362dfe24c83d31f82b", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/stress/com_github_cockroachdb_stress-v0.0.0-20220803192808-1806698b1b7b.zip": "3fda531795c600daf25532a4f98be2a1335cd1e5e182c72789bca79f5f69fcc1", diff --git a/go.mod b/go.mod index 38d4bf4d4c88..48244f7e52c1 100644 --- a/go.mod +++ b/go.mod @@ -135,7 +135,7 @@ require ( github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55 github.com/cockroachdb/gostdlib v1.19.0 github.com/cockroachdb/logtags v0.0.0-20241215232642-bb51bb14a506 - github.com/cockroachdb/pebble v0.0.0-20250111211125-38fb0512c50a + github.com/cockroachdb/pebble v0.0.0-20250113205511-3748221737d4 github.com/cockroachdb/redact v1.1.5 github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd github.com/cockroachdb/stress v0.0.0-20220803192808-1806698b1b7b diff --git a/go.sum b/go.sum index 0796cfdab90c..e6130a05ed58 100644 --- a/go.sum +++ b/go.sum @@ -556,8 +556,8 @@ github.com/cockroachdb/logtags v0.0.0-20241215232642-bb51bb14a506 h1:ASDL+UJcILM github.com/cockroachdb/logtags v0.0.0-20241215232642-bb51bb14a506/go.mod h1:Mw7HqKr2kdtu6aYGn3tPmAftiP3QPX63LdK/zcariIo= github.com/cockroachdb/metamorphic v0.0.0-20231108215700-4ba948b56895 h1:XANOgPYtvELQ/h4IrmPAohXqe2pWA8Bwhejr3VQoZsA= github.com/cockroachdb/metamorphic v0.0.0-20231108215700-4ba948b56895/go.mod h1:aPd7gM9ov9M8v32Yy5NJrDyOcD8z642dqs+F0CeNXfA= -github.com/cockroachdb/pebble v0.0.0-20250111211125-38fb0512c50a h1:eJJFy22HbaaWZpxgQDZi7sOyy2B8M3kYFfzwVJr2z3M= -github.com/cockroachdb/pebble v0.0.0-20250111211125-38fb0512c50a/go.mod h1:ewJSTQ30qIuX6FeYX+2M37Ghn6r0r2I+g0jDIcTdUXM= +github.com/cockroachdb/pebble v0.0.0-20250113205511-3748221737d4 h1:WnGNMXYEsv7uJbHGkl025sXFbjlXaXck5hQtZ0hCZRA= +github.com/cockroachdb/pebble v0.0.0-20250113205511-3748221737d4/go.mod h1:ewJSTQ30qIuX6FeYX+2M37Ghn6r0r2I+g0jDIcTdUXM= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/redact v1.1.5 h1:u1PMllDkdFfPWaNGMyLD1+so+aq3uUItthCFqzwPJ30= github.com/cockroachdb/redact v1.1.5/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= diff --git a/pkg/backup/testdata/backup-restore/restore-validation-only b/pkg/backup/testdata/backup-restore/restore-validation-only index eb5c64c3cc5b..8485fe45ce98 100644 --- a/pkg/backup/testdata/backup-restore/restore-validation-only +++ b/pkg/backup/testdata/backup-restore/restore-validation-only @@ -169,7 +169,7 @@ RESTORE DATABASE d FROM LATEST IN 'nodelocal://1/full_database_backup/' with sch ---- # But verify_backup_table_data catches the corrupt backup file -exec-sql expect-error-regex=(pebble/table: invalid table 000000) +exec-sql expect-error-regex=(checksum mismatch) RESTORE DATABASE d FROM LATEST IN 'nodelocal://1/full_database_backup/' with schema_only, verify_backup_table_data, new_db_name='d4'; ---- regex matches error diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index f2179246e8c6..2817ada46754 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -248,7 +248,7 @@ go_library( "@com_github_cockroachdb_pebble//objstorage", "@com_github_cockroachdb_pebble//objstorage/remote", "@com_github_cockroachdb_pebble//rangekey", - "@com_github_cockroachdb_pebble//sstable", + "@com_github_cockroachdb_pebble//sstable/block", "@com_github_cockroachdb_pebble//vfs", "@com_github_cockroachdb_redact//:redact", "@com_github_gogo_protobuf//proto", diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index 4611d96e8d7e..06de7ae64f40 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -30,7 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/metric/aggmetric" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/pebble" - "github.com/cockroachdb/pebble/sstable" + "github.com/cockroachdb/pebble/sstable/block" "github.com/cockroachdb/pebble/vfs" ) @@ -4161,7 +4161,7 @@ type pebbleCategoryIterMetrics struct { IterBlockReadLatencySum *metric.Counter } -func makePebbleCategorizedIterMetrics(category sstable.Category) pebbleCategoryIterMetrics { +func makePebbleCategorizedIterMetrics(category block.Category) pebbleCategoryIterMetrics { metaBlockBytes := metric.Metadata{ Name: fmt.Sprintf("storage.iterator.category-%s.block-load.bytes", category), Help: "Bytes loaded by storage sstable iterators (possibly cached).", @@ -4190,7 +4190,7 @@ func makePebbleCategorizedIterMetrics(category sstable.Category) pebbleCategoryI // MetricStruct implements the metric.Struct interface. func (m *pebbleCategoryIterMetrics) MetricStruct() {} -func (m *pebbleCategoryIterMetrics) update(stats sstable.CategoryStats) { +func (m *pebbleCategoryIterMetrics) update(stats block.CategoryStats) { m.IterBlockBytes.Update(int64(stats.BlockBytes)) m.IterBlockBytesInCache.Update(int64(stats.BlockBytesInCache)) m.IterBlockReadLatencySum.Update(int64(stats.BlockReadDuration)) @@ -4198,20 +4198,20 @@ func (m *pebbleCategoryIterMetrics) update(stats sstable.CategoryStats) { type pebbleCategoryIterMetricsContainer struct { registry *metric.Registry - // metrics slice for all categories; can be directly indexed by sstable.Category. + // metrics slice for all categories; can be directly indexed by block.Category. metrics []pebbleCategoryIterMetrics } func (m *pebbleCategoryIterMetricsContainer) init(registry *metric.Registry) { m.registry = registry - categories := sstable.Categories() + categories := block.Categories() m.metrics = make([]pebbleCategoryIterMetrics, len(categories)) for _, c := range categories { m.metrics[c] = makePebbleCategorizedIterMetrics(c) } } -func (m *pebbleCategoryIterMetricsContainer) update(stats []sstable.CategoryStatsAggregate) { +func (m *pebbleCategoryIterMetricsContainer) update(stats []block.CategoryStatsAggregate) { for _, s := range stats { m.metrics[s.Category].update(s.CategoryStats) } diff --git a/pkg/storage/fs/BUILD.bazel b/pkg/storage/fs/BUILD.bazel index 6be281be5fb4..9cb59c5705ad 100644 --- a/pkg/storage/fs/BUILD.bazel +++ b/pkg/storage/fs/BUILD.bazel @@ -30,7 +30,7 @@ go_library( "@com_github_cockroachdb_errors//oserror", "@com_github_cockroachdb_pebble//:pebble", "@com_github_cockroachdb_pebble//record", - "@com_github_cockroachdb_pebble//sstable", + "@com_github_cockroachdb_pebble//sstable/block", "@com_github_cockroachdb_pebble//vfs", "@com_github_cockroachdb_pebble//vfs/atomicfs", "@com_github_cockroachdb_pebble//vfs/vfstest", diff --git a/pkg/storage/fs/category.go b/pkg/storage/fs/category.go index 44388a239eaa..2e63b2457dd5 100644 --- a/pkg/storage/fs/category.go +++ b/pkg/storage/fs/category.go @@ -6,7 +6,7 @@ package fs import ( - "github.com/cockroachdb/pebble/sstable" + "github.com/cockroachdb/pebble/sstable/block" "github.com/cockroachdb/pebble/vfs" ) @@ -46,23 +46,23 @@ const ( BackupReadCategory ) -var readCategoryMap = [...]sstable.Category{ - UnknownReadCategory: sstable.RegisterCategory("crdb-unknown", sstable.LatencySensitiveQoSLevel), +var readCategoryMap = [...]block.Category{ + UnknownReadCategory: block.RegisterCategory("crdb-unknown", block.LatencySensitiveQoSLevel), // TODO(sumeer): consider splitting batch-eval into two categories, for // latency sensitive and non latency sensitive. - BatchEvalReadCategory: sstable.RegisterCategory("batch-eval", sstable.LatencySensitiveQoSLevel), - ScanRegularBatchEvalReadCategory: sstable.RegisterCategory("scan-regular", sstable.LatencySensitiveQoSLevel), - ScanBackgroundBatchEvalReadCategory: sstable.RegisterCategory("scan-background", sstable.NonLatencySensitiveQoSLevel), - MVCCGCReadCategory: sstable.RegisterCategory("mvcc-gc", sstable.NonLatencySensitiveQoSLevel), - RangeSnapshotReadCategory: sstable.RegisterCategory("range-snap", sstable.NonLatencySensitiveQoSLevel), - RangefeedReadCategory: sstable.RegisterCategory("rangefeed", sstable.LatencySensitiveQoSLevel), - ReplicationReadCategory: sstable.RegisterCategory("replication", sstable.LatencySensitiveQoSLevel), - IntentResolutionReadCategory: sstable.RegisterCategory("intent-resolution", sstable.LatencySensitiveQoSLevel), - BackupReadCategory: sstable.RegisterCategory("backup", sstable.NonLatencySensitiveQoSLevel), + BatchEvalReadCategory: block.RegisterCategory("batch-eval", block.LatencySensitiveQoSLevel), + ScanRegularBatchEvalReadCategory: block.RegisterCategory("scan-regular", block.LatencySensitiveQoSLevel), + ScanBackgroundBatchEvalReadCategory: block.RegisterCategory("scan-background", block.NonLatencySensitiveQoSLevel), + MVCCGCReadCategory: block.RegisterCategory("mvcc-gc", block.NonLatencySensitiveQoSLevel), + RangeSnapshotReadCategory: block.RegisterCategory("range-snap", block.NonLatencySensitiveQoSLevel), + RangefeedReadCategory: block.RegisterCategory("rangefeed", block.LatencySensitiveQoSLevel), + ReplicationReadCategory: block.RegisterCategory("replication", block.LatencySensitiveQoSLevel), + IntentResolutionReadCategory: block.RegisterCategory("intent-resolution", block.LatencySensitiveQoSLevel), + BackupReadCategory: block.RegisterCategory("backup", block.NonLatencySensitiveQoSLevel), } -// PebbleCategory returns the sstable.Category associated with the given ReadCategory. -func (c ReadCategory) PebbleCategory() sstable.Category { +// PebbleCategory returns the block.Category associated with the given ReadCategory. +func (c ReadCategory) PebbleCategory() block.Category { return readCategoryMap[c] } diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 11510a73777c..a37bf7f79bcc 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1711,7 +1711,7 @@ func (p *Pebble) ScanInternal( rawLower := EngineKey{Key: lower}.Encode() rawUpper := EngineKey{Key: upper}.Encode() // TODO(sumeer): set category. - return p.db.ScanInternal(ctx, sstable.CategoryUnknown, rawLower, rawUpper, visitPointKey, + return p.db.ScanInternal(ctx, block.CategoryUnknown, rawLower, rawUpper, visitPointKey, visitRangeDel, visitRangeKey, visitSharedFile, visitExternalFile) } @@ -3108,7 +3108,7 @@ func (p *pebbleSnapshot) ScanInternal( rawLower := EngineKey{Key: lower}.Encode() rawUpper := EngineKey{Key: upper}.Encode() // TODO(sumeer): set category. - return p.snapshot.ScanInternal(ctx, sstable.CategoryUnknown, rawLower, rawUpper, visitPointKey, + return p.snapshot.ScanInternal(ctx, block.CategoryUnknown, rawLower, rawUpper, visitPointKey, visitRangeDel, visitRangeKey, visitSharedFile, visitExternalFile) } @@ -3232,7 +3232,7 @@ func (p *pebbleEFOS) ScanInternal( rawLower := EngineKey{Key: lower}.Encode() rawUpper := EngineKey{Key: upper}.Encode() // TODO(sumeer): set category. - return p.efos.ScanInternal(ctx, sstable.CategoryUnknown, rawLower, rawUpper, visitPointKey, + return p.efos.ScanInternal(ctx, block.CategoryUnknown, rawLower, rawUpper, visitPointKey, visitRangeDel, visitRangeKey, visitSharedFile, visitExternalFile) } diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 2d50bbbe127b..3b183d704bb4 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -1709,8 +1709,8 @@ func TestPebbleLoggingSlowReads(t *testing.T) { slowCount := testFunc(t, "pebble_logger_and_tracer") require.Equal(t, 0, slowCount) }) - t.Run("reader", func(t *testing.T) { - slowCount := testFunc(t, "reader") + t.Run("block", func(t *testing.T) { + slowCount := testFunc(t, "block") require.Less(t, 0, slowCount) }) } From 65519b975ab1cb4d83dcde8970a006d0ceda7f70 Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Tue, 14 Jan 2025 19:02:35 -0600 Subject: [PATCH 114/126] build: disable the use of PGO in release, `roachtest`, `--cross` builds Epic: CRDB-41952 Release note (build change): Roll back the use of PGO for releases. We plan to implement this instead for 25.2. --- .../nightlies/roachtest_compile_component.sh | 4 +- dev | 2 +- pkg/cmd/dev/build.go | 2 +- pkg/cmd/publish-artifacts/main_test.go | 114 +++++++-------- .../main_test.go | 130 +++++++++--------- pkg/release/build.go | 2 +- 6 files changed, 127 insertions(+), 127 deletions(-) diff --git a/build/teamcity/cockroach/nightlies/roachtest_compile_component.sh b/build/teamcity/cockroach/nightlies/roachtest_compile_component.sh index 9a0996ca4e44..2d1eca9d0fca 100755 --- a/build/teamcity/cockroach/nightlies/roachtest_compile_component.sh +++ b/build/teamcity/cockroach/nightlies/roachtest_compile_component.sh @@ -66,12 +66,12 @@ artifacts=() case "$component" in cockroach) # Cockroach binary. - bazel_args=(--config force_build_cdeps --config pgo --norun_validations //pkg/cmd/cockroach $crdb_extra_flags) + bazel_args=(--config force_build_cdeps --norun_validations //pkg/cmd/cockroach $crdb_extra_flags) artifacts=("pkg/cmd/cockroach/cockroach_/cockroach:bin/cockroach.$os-$arch") ;; cockroach-ea) # Cockroach binary with enabled assertions (EA). - bazel_args=(--config force_build_cdeps --config pgo --norun_validations //pkg/cmd/cockroach --crdb_test $crdb_extra_flags) + bazel_args=(--config force_build_cdeps --norun_validations //pkg/cmd/cockroach --crdb_test $crdb_extra_flags) artifacts=("pkg/cmd/cockroach/cockroach_/cockroach:bin/cockroach-ea.$os-$arch") ;; workload) diff --git a/dev b/dev index ece94bba387d..58c80eaafb9d 100755 --- a/dev +++ b/dev @@ -8,7 +8,7 @@ fi set -euo pipefail # Bump this counter to force rebuilding `dev` on all machines. -DEV_VERSION=106 +DEV_VERSION=107 THIS_DIR=$(cd "$(dirname "$0")" && pwd) BINARY_DIR=$THIS_DIR/bin/dev-versions diff --git a/pkg/cmd/dev/build.go b/pkg/cmd/dev/build.go index 870aaf67b38b..1ee15fbd44a4 100644 --- a/pkg/cmd/dev/build.go +++ b/pkg/cmd/dev/build.go @@ -188,7 +188,7 @@ func (d *dev) crossBuild( volume string, dockerArgs []string, ) error { - bazelArgs = append(bazelArgs, fmt.Sprintf("--config=%s", crossConfig), "--config=nolintonbuild", "-c", "opt", "--config=pgo") + bazelArgs = append(bazelArgs, fmt.Sprintf("--config=%s", crossConfig), "--config=nolintonbuild", "-c", "opt") configArgs := getConfigArgs(bazelArgs) dockerArgs, err := d.getDockerRunArgs(ctx, volume, false, dockerArgs) if err != nil { diff --git a/pkg/cmd/publish-artifacts/main_test.go b/pkg/cmd/publish-artifacts/main_test.go index 497f77195fd6..bb0798d8bd88 100644 --- a/pkg/cmd/publish-artifacts/main_test.go +++ b/pkg/cmd/publish-artifacts/main_test.go @@ -114,7 +114,7 @@ func (r *mockExecRunner) run(c *exec.Cmd) ([]byte, error) { platform = release.PlatformWindows path += ".exe" pathSQL += ".exe" - case "ci", "force_build_cdeps", "pgo": + case "ci", "force_build_cdeps": default: panic(fmt.Sprintf("Unexpected configuration %s", arg)) } @@ -158,128 +158,128 @@ func TestPublish(t *testing.T) { }, expectedCmds: []string{ "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64 version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxbase --norun_validations", "env=[] args=bazel info bazel-bin -c opt --config=crosslinuxbase", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxfipsbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64-fips version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64-fips", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64-fips", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxarmbase", "env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxarmbase --norun_validations", "env=[] args=bazel info bazel-bin -c opt --config=crosslinuxarmbase", "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crossmacosbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=crossmacosarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crossmacosarmbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary' -c opt --config=force_build_cdeps --config=crosswindowsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosswindowsbase", }, expectedPuts: []string{ "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach.linux-gnu-amd64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-amd64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-amd64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-amd64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/workload.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/workload.LATEST/no-cache REDIRECT /cockroach/workload.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64-fips.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64-fips.LATEST/no-cache REDIRECT /cockroach/cockroach.linux-gnu-amd64-fips.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64-fips.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64-fips.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-amd64-fips.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64-fips.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64-fips.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-amd64-fips.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64-fips.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64-fips.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-amd64-fips.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-arm64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-arm64.LATEST/no-cache REDIRECT /cockroach/cockroach.linux-gnu-arm64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-arm64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-arm64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-arm64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-arm64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-arm64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-arm64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-arm64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-arm64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-arm64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/workload.linux-gnu-arm64.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/workload.linux-gnu-arm64.LATEST/no-cache REDIRECT /cockroach/workload.linux-gnu-arm64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach.darwin-amd64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.darwin-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach.darwin-amd64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-amd64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.darwin-amd64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/lib/libgeos.darwin-amd64.1234567890abcdef.dylib CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.darwin-amd64.dylib.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.darwin-amd64.1234567890abcdef.dylib", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.darwin-amd64.1234567890abcdef.dylib CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.darwin-amd64.dylib.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.darwin-amd64.1234567890abcdef.dylib", - "gs://edge-binaries-bucket/cockroach/cockroach.darwin-arm64.unsigned.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach.darwin-arm64.unsigned.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=crossmacosarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.darwin-arm64.unsigned.LATEST/no-cache REDIRECT /cockroach/cockroach.darwin-arm64.unsigned.1234567890abcdef", - "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-arm64.unsigned.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-arm64.unsigned.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=crossmacosarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-arm64.unsigned.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.darwin-arm64.unsigned.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach.windows-amd64.1234567890abcdef.exe CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosswindowsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.windows-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach.windows-amd64.1234567890abcdef.exe", "gs://edge-binaries-bucket/cockroach/cockroach-sql.windows-amd64.1234567890abcdef.exe CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosswindowsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.windows-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.windows-amd64.1234567890abcdef.exe", }, platforms: release.DefaultPlatforms(), @@ -292,8 +292,8 @@ func TestPublish(t *testing.T) { }, expectedCmds: []string{ "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64 version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64", @@ -304,22 +304,22 @@ func TestPublish(t *testing.T) { "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach.linux-gnu-amd64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-amd64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-amd64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-amd64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/workload.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/workload.LATEST/no-cache REDIRECT /cockroach/workload.1234567890abcdef", @@ -334,20 +334,20 @@ func TestPublish(t *testing.T) { }, expectedCmds: []string{ "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64 version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxbase --norun_validations", "env=[] args=bazel info bazel-bin -c opt --config=crosslinuxbase", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxfipsbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64-fips version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64-fips", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64-fips", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxarmbase", "env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxarmbase --norun_validations", "env=[] args=bazel info bazel-bin -c opt --config=crosslinuxarmbase", }, @@ -355,64 +355,64 @@ func TestPublish(t *testing.T) { "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach.linux-gnu-amd64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-amd64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-amd64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-amd64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/workload.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/workload.LATEST/no-cache REDIRECT /cockroach/workload.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64-fips.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64-fips.LATEST/no-cache REDIRECT /cockroach/cockroach.linux-gnu-amd64-fips.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64-fips.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64-fips.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-amd64-fips.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64-fips.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64-fips.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-amd64-fips.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64-fips.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64-fips.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-amd64-fips.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-arm64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-arm64.LATEST/no-cache REDIRECT /cockroach/cockroach.linux-gnu-arm64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-arm64.1234567890abcdef CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-arm64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-arm64.1234567890abcdef", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-arm64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-arm64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-arm64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-arm64.1234567890abcdef.so CONTENTS env=[] args=bazel build " + "//pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' " + - "-c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "-c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-arm64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-arm64.1234567890abcdef.so", "gs://edge-binaries-bucket/cockroach/workload.linux-gnu-arm64.1234567890abcdef CONTENTS env=[] args=bazel build //pkg/cmd/workload -c opt --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/workload.linux-gnu-arm64.LATEST/no-cache REDIRECT /cockroach/workload.linux-gnu-arm64.1234567890abcdef", diff --git a/pkg/cmd/publish-provisional-artifacts/main_test.go b/pkg/cmd/publish-provisional-artifacts/main_test.go index 24b9cb4609bb..5c1d6ad703e7 100644 --- a/pkg/cmd/publish-provisional-artifacts/main_test.go +++ b/pkg/cmd/publish-provisional-artifacts/main_test.go @@ -124,7 +124,7 @@ func (r *mockExecRunner) run(c *exec.Cmd) ([]byte, error) { platform = release.PlatformWindows path += ".exe" pathSQL += ".exe" - case "ci", "force_build_cdeps", "pgo": + case "ci", "force_build_cdeps": default: panic(fmt.Sprintf("Unexpected configuration %s", arg)) } @@ -172,27 +172,27 @@ func TestProvisional(t *testing.T) { }, expectedCmds: []string{ "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64 version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary release' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxfipsbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64-fips version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64-fips", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64-fips", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxarmbase", "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary release' -c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crossmacosbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary release' -c opt --config=force_build_cdeps --config=crossmacosarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crossmacosarmbase", "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles " + "'--workspace_status_command=." + - "/build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase", + "/build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary release' -c opt --config=force_build_cdeps --config=crosswindowsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosswindowsbase", }, expectedGets: nil, expectedPuts: []string{ @@ -233,8 +233,8 @@ func TestProvisional(t *testing.T) { }, expectedCmds: []string{ "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64 version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64", @@ -258,18 +258,18 @@ func TestProvisional(t *testing.T) { }, expectedCmds: []string{ "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64 version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary release' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxfipsbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64-fips version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64-fips", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64-fips", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary release' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxarmbase", }, expectedGets: nil, expectedPuts: []string{ @@ -299,27 +299,27 @@ func TestProvisional(t *testing.T) { }, expectedCmds: []string{ "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary release injected-tag' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary release injected-tag' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64 version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary release injected-tag' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary release injected-tag' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxfipsbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64-fips version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64-fips", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64-fips", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary release injected-tag' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary release injected-tag' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxarmbase", "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary release injected-tag' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary release injected-tag' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary release injected-tag' -c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crossmacosbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary release injected-tag' -c opt --config=force_build_cdeps --config=crossmacosarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crossmacosarmbase", "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles " + "'--workspace_status_command=." + - "/build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary release injected-tag' -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase", + "/build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary release injected-tag' -c opt --config=force_build_cdeps --config=crosswindowsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosswindowsbase", }, expectedGets: nil, expectedPuts: []string{ @@ -362,99 +362,99 @@ func TestProvisional(t *testing.T) { expectedCmds: []string{ "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=." + - "/build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase", + "/build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64 version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxfipsbase", "env=[MALLOC_CONF=prof:true] args=./cockroach.linux-2.6.32-gnu-amd64-fips version", "env=[] args=ldd ./cockroach.linux-2.6.32-gnu-amd64-fips", "env=[] args=bazel run @go_sdk//:bin/go -- tool nm ./cockroach.linux-2.6.32-gnu-amd64-fips", "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosslinuxarmbase", "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase", - "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config" + + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crossmacosbase", + "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=crossmacosarmbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config" + "=crossmacosarmbase", "env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase --norun_validations", - "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary' -c opt --config=force_build_cdeps --config=crosswindowsbase --norun_validations", + "env=[] args=bazel info bazel-bin -c opt --config=force_build_cdeps --config=crosswindowsbase", }, expectedGets: nil, expectedPuts: []string{ "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64.00SHA00 " + "CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp." + - "sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64.LATEST/no-cache " + "REDIRECT /cockroach/cockroach.linux-gnu-amd64.00SHA00", - "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-amd64.00SHA00", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64.00SHA00." + "so CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-amd64.00SHA00.so", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64.00SHA00." + "so CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxbase --norun_validations", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-amd64.00SHA00.so", - "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64-fips.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64-fips.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-amd64-fips.LATEST/no-cache REDIRECT /cockroach/cockroach.linux-gnu-amd64-fips.00SHA00", - "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64-fips.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64-fips.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-amd64-fips.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-amd64-fips.00SHA00", - "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64-fips.00SHA00.so CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64-fips.00SHA00.so CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-amd64-fips.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-amd64-fips.00SHA00.so", - "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64-fips.00SHA00.so CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxfipsbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64-fips.00SHA00.so CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-pc-linux-gnu official-fips-binary' -c opt --config=force_build_cdeps --config=crosslinuxfipsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-amd64-fips.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-amd64-fips.00SHA00.so", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-arm64.00SHA00 " + "CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp." + - "sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.linux-gnu-arm64.LATEST/no-cache " + "REDIRECT /cockroach/cockroach.linux-gnu-arm64.00SHA00", - "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-arm64.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-arm64.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.linux-gnu-arm64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.linux-gnu-arm64.00SHA00", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-arm64.00SHA00." + "so CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.linux-gnu-arm64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.linux-gnu-arm64.00SHA00.so", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-arm64.00SHA00." + "so CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosslinuxarmbase --norun_validations", + "'--workspace_status_command=./build/bazelutil/stamp.sh aarch64-unknown-linux-gnu official-binary' -c opt --config=force_build_cdeps --config=crosslinuxarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.linux-gnu-arm64.so.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.linux-gnu-arm64.00SHA00.so", "gs://edge-binaries-bucket/cockroach/cockroach.darwin-amd64.00SHA00 " + "CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.darwin-amd64.LATEST/no-cache " + "REDIRECT /cockroach/cockroach.darwin-amd64.00SHA00", - "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-amd64.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-amd64.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql." + "darwin-amd64.00SHA00", "gs://edge-binaries-bucket/cockroach/lib/libgeos.darwin-amd64.00SHA00." + "dylib CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + - "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", + "'--workspace_status_command=./build/bazelutil/stamp.sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos.darwin-amd64.dylib.LATEST/no-cache REDIRECT /cockroach/lib/libgeos.darwin-amd64.00SHA00.dylib", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.darwin-amd64.00SHA00." + "dylib CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql //c-deps:libgeos " + "'--workspace_status_command=./build/bazelutil/stamp." + - "sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosbase --norun_validations", + "sh x86_64-apple-darwin19 official-binary' -c opt --config=force_build_cdeps --config=crossmacosbase --norun_validations", "gs://edge-binaries-bucket/cockroach/lib/libgeos_c.darwin-amd64.dylib.LATEST/no-cache REDIRECT /cockroach/lib/libgeos_c.darwin-amd64.00SHA00.dylib", - "gs://edge-binaries-bucket/cockroach/cockroach.darwin-arm64.unsigned.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach.darwin-arm64.unsigned.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=crossmacosarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.darwin-arm64.unsigned.LATEST/no-cache REDIRECT /cockroach/cockroach.darwin-arm64.unsigned.00SHA00", - "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-arm64.unsigned.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crossmacosarmbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-arm64.unsigned.00SHA00 CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql '--workspace_status_command=./build/bazelutil/stamp.sh aarch64-apple-darwin21.2 official-binary' -c opt --config=force_build_cdeps --config=crossmacosarmbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.darwin-arm64.unsigned.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.darwin-arm64.unsigned.00SHA00", "gs://edge-binaries-bucket/cockroach/cockroach.windows-amd64.00SHA00.exe " + "CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles " + "'--workspace_status_command=./build/bazelutil/stamp." + - "sh x86_64-w64-mingw32 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase --norun_validations", + "sh x86_64-w64-mingw32 official-binary' -c opt --config=force_build_cdeps --config=crosswindowsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach.windows-amd64.LATEST/no-cache " + "REDIRECT /cockroach/cockroach.windows-amd64.00SHA00.exe", - "gs://edge-binaries-bucket/cockroach/cockroach-sql.windows-amd64.00SHA00.exe CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary' -c opt --config=force_build_cdeps --config=pgo --config=crosswindowsbase --norun_validations", + "gs://edge-binaries-bucket/cockroach/cockroach-sql.windows-amd64.00SHA00.exe CONTENTS env=[] args=bazel build //pkg/cmd/cockroach //pkg/cmd/cockroach-sql --enable_runfiles '--workspace_status_command=./build/bazelutil/stamp.sh x86_64-w64-mingw32 official-binary' -c opt --config=force_build_cdeps --config=crosswindowsbase --norun_validations", "gs://edge-binaries-bucket/cockroach/cockroach-sql.windows-amd64.LATEST/no-cache REDIRECT /cockroach/cockroach-sql.windows-amd64.00SHA00.exe", }, platforms: release.DefaultPlatforms(), diff --git a/pkg/release/build.go b/pkg/release/build.go index 5c8b12675985..3b36a2f336ca 100644 --- a/pkg/release/build.go +++ b/pkg/release/build.go @@ -176,7 +176,7 @@ func MakeRelease(platform Platform, opts BuildOptions, pkgDir string) error { stampCommand = fmt.Sprintf("--workspace_status_command=./build/bazelutil/stamp.sh %s %s", targetTriple, opts.Channel) } buildArgs = append(buildArgs, stampCommand) - configs := []string{"-c", "opt", "--config=force_build_cdeps", "--config=pgo", fmt.Sprintf("--config=%s", CrossConfigFromPlatform(platform))} + configs := []string{"-c", "opt", "--config=force_build_cdeps", fmt.Sprintf("--config=%s", CrossConfigFromPlatform(platform))} buildArgs = append(buildArgs, configs...) buildArgs = append(buildArgs, "--norun_validations") cmd := exec.Command("bazel", buildArgs...) From d2643068c289f67e6fa167338d6ec372d9924238 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 13 Jan 2025 17:44:29 -0800 Subject: [PATCH 115/126] colexec: fix memory leak in simpleProjectOp This commit fixes a bounded memory leak in `simpleProjectOp` that has been present since 20.2 version. The leak was introduced via the combination of - 57eb4f8528491169c1ec14888c2cca448bf7df02 in which we began tracking all batches seen by the operator so that we could decide whether we need to allocate a fresh projecting batch or not - 895125bc90ce0d286eeabb49e4b7cf7a75322045 in which we started using the dynamic sizing for batches (where we'd start with size 1 and grow exponentially until 1024 while previously we would always use 1024). Both changes combined made it so that the `simpleProjectOp` would keep _all_ batches with different sizes alive until the query shutdown. The problem later got more exacerbated when we introduced dynamic batch sizing all over the place (for example, in the spilling queue). Let's discuss the reasoning for why we needed something like the tracking we did in the first change mentioned above. The simple project op works by putting a small wrapper (a "lense") `projectingBatch` over the batch coming from the input. That wrapper can be modified later by other operators (for example, a new vector might be appended), which would also modify the original batch coming from the input, so we need to allow for the wrapper to be mutated accordingly. At the same time, the input operator might decide to produce a fresh batch (for example, because of the dynamically growing size) which wouldn't have the same "upstream" mutation applied to it, so we need to allocate a fresh projecting batch and let the upstream do its modifications. In the first change we solved it by keeping a map from the input batch to the projecting batch. This commit addresses the same issue by only checking whether the input batch is the same one as was seen by the `simpleProjectOp` on the previous call. If they are the same, then we can reuse the last projecting batch; otherwise, we need to allocate a fresh one and memoize it. In other words, we effectively reduce the map to have at most one entry. This means that with dynamic batch size growth we'd create a few `projectingBatch` wrappers, but that is ok given that we expect the dynamic size heuristics to quickly settle on the size. This commit adjusts the contract of `Operator.Next` to explicitly mention that implementations should reuse the same batch whenever possible. This is already the case pretty much everywhere, except when the dynamic batch size grows or shrinks. Before we introduced the dynamic batch sizing, different batches could only be produced by the unordered synchronizers, so that's another place this commit adjusts. If we didn't do anything, then the simplistic mapping with at most one entry could result in "thrashing" - i.e. in the extreme case where the inputs to the synchronizer would produce batches in round-robin fashion, we'd end up creating a new `projectingBatch` every time which would be quite wasteful. In this commit we modify both the parallel and the serial unordered synchronizers to always emit the same output batch which is populated by manually inserting vectors from the input batch. I did some manual testing on the impact of this change. I used a table and a query (with some window functions) from the customer cluster that has been seeing some OOMs. I had one node cluster running locally with `--max-go-memory=256MiB` (so that the memory needed by the query was forcing GC all the time since it exceeded the soft memory limit) and `distsql_workmem=128MiB` (since we cannot spill to disk some state in the row-by-row window functions). Before this patch I observed the max RAM usage at 1.42GB, and after this patch 0.56GB (the latter is pretty close to what we report on EXPLAIN ANALYZE). Release note (bug fix): Bounded memory leak that could previously occur when evaluating some memory-intensive queries via the vectorized engine in CockroachDB has now been fixed. The leak has been present since 20.2 version. --- pkg/sql/colexec/colexecbase/BUILD.bazel | 2 +- pkg/sql/colexec/colexecbase/cast.eg.go | 1 + pkg/sql/colexec/colexecbase/cast_tmpl.go | 1 + pkg/sql/colexec/colexecbase/simple_project.go | 40 +++++++++---------- .../colexecbase/simple_project_test.go | 2 +- .../parallel_unordered_synchronizer.go | 22 ++++++++-- .../parallel_unordered_synchronizer_test.go | 9 +++-- .../colexec/serial_unordered_synchronizer.go | 19 ++++++++- .../serial_unordered_synchronizer_test.go | 2 + pkg/sql/colexecop/operator.go | 3 ++ pkg/sql/colflow/vectorized_flow.go | 20 +++++----- .../colflow/vectorized_flow_shutdown_test.go | 2 +- 12 files changed, 81 insertions(+), 42 deletions(-) diff --git a/pkg/sql/colexec/colexecbase/BUILD.bazel b/pkg/sql/colexec/colexecbase/BUILD.bazel index 2d94743e6e24..1eaeca8b100f 100644 --- a/pkg/sql/colexec/colexecbase/BUILD.bazel +++ b/pkg/sql/colexec/colexecbase/BUILD.bazel @@ -33,7 +33,7 @@ go_library( "//pkg/util", # keep "//pkg/util/duration", # keep "//pkg/util/json", # keep - "//pkg/util/log", + "//pkg/util/log", # keep "//pkg/util/timeutil/pgdate", # keep "//pkg/util/uuid", # keep "@com_github_cockroachdb_apd_v3//:apd", # keep diff --git a/pkg/sql/colexec/colexecbase/cast.eg.go b/pkg/sql/colexec/colexecbase/cast.eg.go index 3a47a0e40af1..7aee514af014 100644 --- a/pkg/sql/colexec/colexecbase/cast.eg.go +++ b/pkg/sql/colexec/colexecbase/cast.eg.go @@ -51,6 +51,7 @@ var ( _ = pgcode.Syntax _ = pgdate.ParseTimestamp _ = pgerror.Wrapf + _ = log.ExpensiveLogEnabled ) func isIdentityCast(fromType, toType *types.T) bool { diff --git a/pkg/sql/colexec/colexecbase/cast_tmpl.go b/pkg/sql/colexec/colexecbase/cast_tmpl.go index 94d5744b9d51..d2255ce32156 100644 --- a/pkg/sql/colexec/colexecbase/cast_tmpl.go +++ b/pkg/sql/colexec/colexecbase/cast_tmpl.go @@ -57,6 +57,7 @@ var ( _ = pgcode.Syntax _ = pgdate.ParseTimestamp _ = pgerror.Wrapf + _ = log.ExpensiveLogEnabled ) // {{/* diff --git a/pkg/sql/colexec/colexecbase/simple_project.go b/pkg/sql/colexec/colexecbase/simple_project.go index 8fd3f87d5429..8b162055c4d4 100644 --- a/pkg/sql/colexec/colexecbase/simple_project.go +++ b/pkg/sql/colexec/colexecbase/simple_project.go @@ -11,20 +11,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" - "github.com/cockroachdb/cockroach/pkg/util/log" ) // simpleProjectOp is an operator that implements "simple projection" - removal of // columns that aren't needed by later operators. type simpleProjectOp struct { colexecop.NonExplainable - batches map[coldata.Batch]*projectingBatch colexecop.OneInputInitCloserHelper + projection []uint32 - // numBatchesLoggingThreshold is the threshold on the number of items in - // 'batches' map at which we will log a message when a new projectingBatch - // is created. It is growing exponentially. - numBatchesLoggingThreshold int + batch *projectingBatch } var _ colexecop.ClosableOperator = &simpleProjectOp{} @@ -105,10 +101,8 @@ func NewSimpleProjectOp( } } s := &simpleProjectOp{ - OneInputInitCloserHelper: colexecop.MakeOneInputInitCloserHelper(input), - projection: make([]uint32, len(projection)), - batches: make(map[coldata.Batch]*projectingBatch), - numBatchesLoggingThreshold: 128, + OneInputInitCloserHelper: colexecop.MakeOneInputInitCloserHelper(input), + projection: make([]uint32, len(projection)), } // We make a copy of projection to be safe. copy(s.projection, projection) @@ -120,19 +114,21 @@ func (d *simpleProjectOp) Next() coldata.Batch { if batch.Length() == 0 { return coldata.ZeroBatch } - projBatch, found := d.batches[batch] - if !found { - projBatch = newProjectionBatch(d.projection) - d.batches[batch] = projBatch - if len(d.batches) == d.numBatchesLoggingThreshold { - if log.V(1) { - log.Infof(d.Ctx, "simpleProjectOp: size of 'batches' map = %d", len(d.batches)) - } - d.numBatchesLoggingThreshold = d.numBatchesLoggingThreshold * 2 - } + if d.batch == nil || d.batch.Batch != batch { + // Create a fresh projection batch if we haven't created one already or + // if we see a different "internally" batch coming from the input. The + // latter case can happen during dynamically growing the size of the + // batch in the input, and we need to create a fresh projection batch + // since the last one might have been modified higher up in the tree + // (e.g. a vector could have been appended). + // + // The contract of Operator.Next encourages implementations to reuse the + // same batch, so we shouldn't be hitting this case often to make this + // allocation have non-trivial impact. + d.batch = newProjectionBatch(d.projection) } - projBatch.Batch = batch - return projBatch + d.batch.Batch = batch + return d.batch } func (d *simpleProjectOp) Reset(ctx context.Context) { diff --git a/pkg/sql/colexec/colexecbase/simple_project_test.go b/pkg/sql/colexec/colexecbase/simple_project_test.go index 5ccf9218ef55..c16ea62422f0 100644 --- a/pkg/sql/colexec/colexecbase/simple_project_test.go +++ b/pkg/sql/colexec/colexecbase/simple_project_test.go @@ -117,7 +117,7 @@ func TestSimpleProjectOpWithUnorderedSynchronizer(t *testing.T) { for i := range parallelUnorderedSynchronizerInputs { parallelUnorderedSynchronizerInputs[i].Root = inputs[i] } - input = colexec.NewParallelUnorderedSynchronizer(&execinfra.FlowCtx{Local: true, Gateway: true}, 0 /* processorID */, testMemAcc, parallelUnorderedSynchronizerInputs, &wg) + input = colexec.NewParallelUnorderedSynchronizer(&execinfra.FlowCtx{Local: true, Gateway: true}, 0 /* processorID */, testAllocator, inputTypes, parallelUnorderedSynchronizerInputs, &wg) input = colexecbase.NewSimpleProjectOp(input, len(inputTypes), []uint32{0}) return colexecbase.NewConstOp(testAllocator, input, types.Int, constVal, 1) }) diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer.go b/pkg/sql/colexec/parallel_unordered_synchronizer.go index cb225e877349..6ce17eaf4ae7 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer.go @@ -17,10 +17,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfra/execopnode" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -96,6 +98,13 @@ type ParallelUnorderedSynchronizer struct { // the corresponding to it input. nextBatch []func() + // outputBatch is the output batch emitted by the synchronizer. + // + // The contract of Operator.Next encourages emitting the same batch across + // Next calls, so batches coming from the inputs are decomposed into vectors + // which are inserted into this batch. + outputBatch coldata.Batch + state int32 // externalWaitGroup refers to the WaitGroup passed in externally. Since the // ParallelUnorderedSynchronizer spawns goroutines, this allows callers to @@ -159,7 +168,8 @@ func hasParallelUnorderedSync(op execopnode.OpNode) bool { func NewParallelUnorderedSynchronizer( flowCtx *execinfra.FlowCtx, processorID int32, - streamingMemAcc *mon.BoundAccount, + allocator *colmem.Allocator, + inputTypes []*types.T, inputs []colexecargs.OpWithMetaInfo, wg *sync.WaitGroup, ) *ParallelUnorderedSynchronizer { @@ -206,13 +216,14 @@ func NewParallelUnorderedSynchronizer( return &ParallelUnorderedSynchronizer{ flowCtx: flowCtx, processorID: processorID, - streamingMemAcc: streamingMemAcc, + streamingMemAcc: allocator.Acc(), inputs: inputs, cancelInputsOnDrain: cancelInputs, tracingSpans: make([]*tracing.Span, len(inputs)), readNextBatch: readNextBatch, batches: make([]coldata.Batch, len(inputs)), nextBatch: make([]func(), len(inputs)), + outputBatch: allocator.NewMemBatchNoCols(inputTypes, coldata.BatchSize() /* capacity */), externalWaitGroup: wg, internalWaitGroup: &sync.WaitGroup{}, // batchCh is a buffered channel in order to offer non-blocking writes to @@ -441,7 +452,11 @@ func (s *ParallelUnorderedSynchronizer) Next() coldata.Batch { s.bufferedMeta = append(s.bufferedMeta, msg.meta...) continue } - return msg.b + for i, vec := range msg.b.ColVecs() { + s.outputBatch.ReplaceCol(vec, i) + } + colexecutils.UpdateBatchState(s.outputBatch, msg.b.Length(), msg.b.Selection() != nil /* usesSel */, msg.b.Selection()) + return s.outputBatch } } } @@ -565,6 +580,7 @@ func (s *ParallelUnorderedSynchronizer) DrainMeta() []execinfrapb.ProducerMetada // Done. s.setState(parallelUnorderedSynchronizerStateDone) + s.outputBatch = nil bufferedMeta := s.bufferedMeta // Eagerly lose the reference to the metadata since it might be of // non-trivial footprint. diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go index d67c91137f68..2d3e10db0e4f 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go @@ -123,7 +123,7 @@ func TestParallelUnorderedSynchronizer(t *testing.T) { ctx, cancelFn := context.WithCancel(context.Background()) var wg sync.WaitGroup - s := NewParallelUnorderedSynchronizer(&execinfra.FlowCtx{Local: true, Gateway: true}, 0 /* processorID */, testMemAcc, inputs, &wg) + s := NewParallelUnorderedSynchronizer(&execinfra.FlowCtx{Local: true, Gateway: true}, 0 /* processorID */, testAllocator, typs, inputs, &wg) s.Init(ctx) t.Run(fmt.Sprintf("numInputs=%d/numBatches=%d/terminationScenario=%d", numInputs, numBatches, terminationScenario), func(t *testing.T) { @@ -193,6 +193,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { // This code is unreachable, but the compiler cannot infer that. return nil }} + typs := types.OneIntCol for i := 1; i < len(inputs); i++ { acc := testMemMonitor.MakeBoundAccount() defer acc.Close(ctx) @@ -201,7 +202,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { NextCb: func() coldata.Batch { // All inputs that do not encounter an error will continue to return // batches. - b := allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + b := allocator.NewMemBatchWithMaxCapacity(typs) b.SetLength(1) return b }, @@ -224,7 +225,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { } var wg sync.WaitGroup - s := NewParallelUnorderedSynchronizer(&execinfra.FlowCtx{Local: true, Gateway: true}, 0 /* processorID */, testMemAcc, inputs, &wg) + s := NewParallelUnorderedSynchronizer(&execinfra.FlowCtx{Local: true, Gateway: true}, 0 /* processorID */, testAllocator, typs, inputs, &wg) s.Init(ctx) for { if err := colexecerror.CatchVectorizedRuntimeError(func() { _ = s.Next() }); err != nil { @@ -256,7 +257,7 @@ func BenchmarkParallelUnorderedSynchronizer(b *testing.B) { } var wg sync.WaitGroup ctx, cancelFn := context.WithCancel(context.Background()) - s := NewParallelUnorderedSynchronizer(&execinfra.FlowCtx{Local: true, Gateway: true}, 0 /* processorID */, testMemAcc, inputs, &wg) + s := NewParallelUnorderedSynchronizer(&execinfra.FlowCtx{Local: true, Gateway: true}, 0 /* processorID */, testAllocator, typs, inputs, &wg) s.Init(ctx) b.SetBytes(8 * int64(coldata.BatchSize())) b.ResetTimer() diff --git a/pkg/sql/colexec/serial_unordered_synchronizer.go b/pkg/sql/colexec/serial_unordered_synchronizer.go index eeb9a1247e87..adf8b1e57b07 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer.go @@ -10,11 +10,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfra/execopnode" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/tracing" ) @@ -30,6 +33,13 @@ type SerialUnorderedSynchronizer struct { processorID int32 span *tracing.Span + // outputBatch is the output batch emitted by the synchronizer. + // + // The contract of Operator.Next encourages emitting the same batch across + // Next calls, so batches coming from the inputs are decomposed into vectors + // which are inserted into this batch. + outputBatch coldata.Batch + inputs []colexecargs.OpWithMetaInfo // curSerialInputIdx indicates the index of the current input being consumed. curSerialInputIdx int @@ -65,6 +75,8 @@ func (s *SerialUnorderedSynchronizer) Child(nth int, verbose bool) execopnode.Op func NewSerialUnorderedSynchronizer( flowCtx *execinfra.FlowCtx, processorID int32, + allocator *colmem.Allocator, + inputTypes []*types.T, inputs []colexecargs.OpWithMetaInfo, serialInputIdxExclusiveUpperBound uint32, exceedsInputIdxExclusiveUpperBoundError error, @@ -72,6 +84,7 @@ func NewSerialUnorderedSynchronizer( return &SerialUnorderedSynchronizer{ flowCtx: flowCtx, processorID: processorID, + outputBatch: allocator.NewMemBatchNoCols(inputTypes, coldata.BatchSize() /* capacity */), inputs: inputs, serialInputIdxExclusiveUpperBound: serialInputIdxExclusiveUpperBound, exceedsInputIdxExclusiveUpperBoundError: exceedsInputIdxExclusiveUpperBoundError, @@ -102,7 +115,11 @@ func (s *SerialUnorderedSynchronizer) Next() coldata.Batch { colexecerror.ExpectedError(s.exceedsInputIdxExclusiveUpperBoundError) } } else { - return b + for i, vec := range b.ColVecs() { + s.outputBatch.ReplaceCol(vec, i) + } + colexecutils.UpdateBatchState(s.outputBatch, b.Length(), b.Selection() != nil /* usesSel */, b.Selection()) + return s.outputBatch } } } diff --git a/pkg/sql/colexec/serial_unordered_synchronizer_test.go b/pkg/sql/colexec/serial_unordered_synchronizer_test.go index ef9f8cc3fee3..5cac57415c2f 100644 --- a/pkg/sql/colexec/serial_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/serial_unordered_synchronizer_test.go @@ -53,6 +53,8 @@ func TestSerialUnorderedSynchronizer(t *testing.T) { s := NewSerialUnorderedSynchronizer( &execinfra.FlowCtx{Gateway: true}, 0, /* processorID */ + testAllocator, + typs, inputs, 0, /* serialInputIdxExclusiveUpperBound */ nil, /* exceedsInputIdxExclusiveUpperBoundError */ diff --git a/pkg/sql/colexecop/operator.go b/pkg/sql/colexecop/operator.go index 2dbb61e9c2a4..eeba25364e05 100644 --- a/pkg/sql/colexecop/operator.go +++ b/pkg/sql/colexecop/operator.go @@ -40,6 +40,9 @@ type Operator interface { // Calling Next may invalidate the contents of the last Batch returned by // Next. // + // Implementations should strive for reusing the same Batch across calls to + // Next which should be possible if the capacity of the Batch didn't change. + // // It might panic with an expected error, so there must be a "root" // component that will catch that panic. Next() coldata.Batch diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index a15e0816fa60..9c7a4f0bc18f 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -953,14 +953,12 @@ func (s *vectorizedFlowCreator) setupInput( opWithMetaInfo := inputStreamOps[0] if len(inputStreamOps) > 1 { statsInputs := inputStreamOps + allocator := colmem.NewAllocator(ctx, s.monitorRegistry.NewStreamingMemAccount(flowCtx), factory) if input.Type == execinfrapb.InputSyncSpec_ORDERED { os := colexec.NewOrderedSynchronizer( - flowCtx, - processorID, - colmem.NewAllocator(ctx, s.monitorRegistry.NewStreamingMemAccount(flowCtx), factory), - execinfra.GetWorkMemLimit(flowCtx), inputStreamOps, - input.ColumnTypes, execinfrapb.ConvertToColumnOrdering(input.Ordering), - 0, /* tuplesToMerge */ + flowCtx, processorID, allocator, execinfra.GetWorkMemLimit(flowCtx), + inputStreamOps, input.ColumnTypes, + execinfrapb.ConvertToColumnOrdering(input.Ordering), 0, /* tuplesToMerge */ ) opWithMetaInfo = colexecargs.OpWithMetaInfo{ Root: os, @@ -975,7 +973,10 @@ func (s *vectorizedFlowCreator) setupInput( err = execinfra.NewDynamicQueryHasNoHomeRegionError(err) } } - sync := colexec.NewSerialUnorderedSynchronizer(flowCtx, processorID, inputStreamOps, input.EnforceHomeRegionStreamExclusiveUpperBound, err) + sync := colexec.NewSerialUnorderedSynchronizer( + flowCtx, processorID, allocator, input.ColumnTypes, inputStreamOps, + input.EnforceHomeRegionStreamExclusiveUpperBound, err, + ) opWithMetaInfo = colexecargs.OpWithMetaInfo{ Root: sync, MetadataSources: colexecop.MetadataSources{sync}, @@ -985,8 +986,9 @@ func (s *vectorizedFlowCreator) setupInput( // Note that if we have opt == flowinfra.FuseAggressively, then we // must use the serial unordered sync above in order to remove any // concurrency. - streamingMemAcc := s.monitorRegistry.NewStreamingMemAccount(flowCtx) - sync := colexec.NewParallelUnorderedSynchronizer(flowCtx, processorID, streamingMemAcc, inputStreamOps, s.f.GetWaitGroup()) + sync := colexec.NewParallelUnorderedSynchronizer( + flowCtx, processorID, allocator, input.ColumnTypes, inputStreamOps, s.f.GetWaitGroup(), + ) opWithMetaInfo = colexecargs.OpWithMetaInfo{ Root: sync, MetadataSources: colexecop.MetadataSources{sync}, diff --git a/pkg/sql/colflow/vectorized_flow_shutdown_test.go b/pkg/sql/colflow/vectorized_flow_shutdown_test.go index 0a72e0129e52..068e1bd41a32 100644 --- a/pkg/sql/colflow/vectorized_flow_shutdown_test.go +++ b/pkg/sql/colflow/vectorized_flow_shutdown_test.go @@ -224,7 +224,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { ) } syncFlowCtx := &execinfra.FlowCtx{Local: false, Gateway: !addAnotherRemote} - synchronizer := colexec.NewParallelUnorderedSynchronizer(syncFlowCtx, 0 /* processorID */, testMemAcc, synchronizerInputs, &wg) + synchronizer := colexec.NewParallelUnorderedSynchronizer(syncFlowCtx, 0 /* processorID */, testAllocator, typs, synchronizerInputs, &wg) inputMetadataSource := colexecop.MetadataSource(synchronizer) runOutboxInbox := func( From 3bd3a33e54c804ba2694fe655f436537aa6e8407 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Tue, 14 Jan 2025 19:25:53 -0800 Subject: [PATCH 116/126] go.mod: bump Pebble to ba0eeec8205a Changes: * [`ba0eeec8`](https://github.com/cockroachdb/pebble/commit/ba0eeec8) vfs: add MemFS.TestingSetDiskUsage * [`8cbd9c6f`](https://github.com/cockroachdb/pebble/commit/8cbd9c6f) sstable: fix row writer mangling of cache block * [`56f4408f`](https://github.com/cockroachdb/pebble/commit/56f4408f) sstable, objstorageprovider: mangle the buffer when writing blocks * [`769e9778`](https://github.com/cockroachdb/pebble/commit/769e9778) db: add a LowDiskSpace event Release note: none. Epic: none. --- DEPS.bzl | 6 +++--- build/bazelutil/distdir_files.bzl | 2 +- go.mod | 2 +- go.sum | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 310df4f5a86f..858874e3b4fa 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -1885,10 +1885,10 @@ def go_deps(): patches = [ "@com_github_cockroachdb_cockroach//build/patches:com_github_cockroachdb_pebble.patch", ], - sha256 = "723fdfb0271c1d50ac9ebb710a9eeb8f249867f918056c0b166c30d13290dc95", - strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20250113205511-3748221737d4", + sha256 = "f91568043e0c5fa6ef1a79043a48c363c523f831330c6efb9c9db208af845422", + strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20250114230814-ba0eeec8205a", urls = [ - "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20250113205511-3748221737d4.zip", + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20250114230814-ba0eeec8205a.zip", ], ) go_repository( diff --git a/build/bazelutil/distdir_files.bzl b/build/bazelutil/distdir_files.bzl index 7de3581c9dfc..e97ef0d1ee16 100644 --- a/build/bazelutil/distdir_files.bzl +++ b/build/bazelutil/distdir_files.bzl @@ -358,7 +358,7 @@ DISTDIR_FILES = { "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/gostdlib/com_github_cockroachdb_gostdlib-v1.19.0.zip": "c4d516bcfe8c07b6fc09b8a9a07a95065b36c2855627cb3514e40c98f872b69e", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/logtags/com_github_cockroachdb_logtags-v0.0.0-20241215232642-bb51bb14a506.zip": "920068af09e3846d9ebb4e4a7787ff1dd10f3989c5f940ad861b0f6a9f824f6e", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/metamorphic/com_github_cockroachdb_metamorphic-v0.0.0-20231108215700-4ba948b56895.zip": "28c8cf42192951b69378cf537be5a9a43f2aeb35542908cc4fe5f689505853ea", - "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20250113205511-3748221737d4.zip": "723fdfb0271c1d50ac9ebb710a9eeb8f249867f918056c0b166c30d13290dc95", + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20250114230814-ba0eeec8205a.zip": "f91568043e0c5fa6ef1a79043a48c363c523f831330c6efb9c9db208af845422", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/redact/com_github_cockroachdb_redact-v1.1.5.zip": "11b30528eb0dafc8bc1a5ba39d81277c257cbe6946a7564402f588357c164560", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/returncheck/com_github_cockroachdb_returncheck-v0.0.0-20200612231554-92cdbca611dd.zip": "ce92ba4352deec995b1f2eecf16eba7f5d51f5aa245a1c362dfe24c83d31f82b", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/stress/com_github_cockroachdb_stress-v0.0.0-20220803192808-1806698b1b7b.zip": "3fda531795c600daf25532a4f98be2a1335cd1e5e182c72789bca79f5f69fcc1", diff --git a/go.mod b/go.mod index 5348ac7046e1..1663a143d51f 100644 --- a/go.mod +++ b/go.mod @@ -136,7 +136,7 @@ require ( github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55 github.com/cockroachdb/gostdlib v1.19.0 github.com/cockroachdb/logtags v0.0.0-20241215232642-bb51bb14a506 - github.com/cockroachdb/pebble v0.0.0-20250113205511-3748221737d4 + github.com/cockroachdb/pebble v0.0.0-20250114230814-ba0eeec8205a github.com/cockroachdb/redact v1.1.5 github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd github.com/cockroachdb/stress v0.0.0-20220803192808-1806698b1b7b diff --git a/go.sum b/go.sum index 4eee7b19025f..cdad6fee5f1e 100644 --- a/go.sum +++ b/go.sum @@ -556,8 +556,8 @@ github.com/cockroachdb/logtags v0.0.0-20241215232642-bb51bb14a506 h1:ASDL+UJcILM github.com/cockroachdb/logtags v0.0.0-20241215232642-bb51bb14a506/go.mod h1:Mw7HqKr2kdtu6aYGn3tPmAftiP3QPX63LdK/zcariIo= github.com/cockroachdb/metamorphic v0.0.0-20231108215700-4ba948b56895 h1:XANOgPYtvELQ/h4IrmPAohXqe2pWA8Bwhejr3VQoZsA= github.com/cockroachdb/metamorphic v0.0.0-20231108215700-4ba948b56895/go.mod h1:aPd7gM9ov9M8v32Yy5NJrDyOcD8z642dqs+F0CeNXfA= -github.com/cockroachdb/pebble v0.0.0-20250113205511-3748221737d4 h1:WnGNMXYEsv7uJbHGkl025sXFbjlXaXck5hQtZ0hCZRA= -github.com/cockroachdb/pebble v0.0.0-20250113205511-3748221737d4/go.mod h1:ewJSTQ30qIuX6FeYX+2M37Ghn6r0r2I+g0jDIcTdUXM= +github.com/cockroachdb/pebble v0.0.0-20250114230814-ba0eeec8205a h1:PUAePGJWmo7UDey3/a3MZfu6SP5rsyPG+GlSK8LA5jM= +github.com/cockroachdb/pebble v0.0.0-20250114230814-ba0eeec8205a/go.mod h1:ewJSTQ30qIuX6FeYX+2M37Ghn6r0r2I+g0jDIcTdUXM= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/redact v1.1.5 h1:u1PMllDkdFfPWaNGMyLD1+so+aq3uUItthCFqzwPJ30= github.com/cockroachdb/redact v1.1.5/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= From fbf1492f0f51c9050b8a5be06185659ef3d5482e Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Tue, 14 Jan 2025 18:32:17 -0500 Subject: [PATCH 117/126] sql: relax validation for IDENTITY columns The validations added in 4f99f9cb8bda4faee94f5d16ff1f1739acb62efa were too strict. This patch makes it so we can continue having IDENTITY columns that are not backed by a sequence. This can happen if a SERIAL column is created with IDENTITY, and serial_normalization=rowid. Release note (bug fix): Fixed a bug where columns created with GENERATED ... BY IDENTITY with the SERIAL type could incorrectly fail internal validations. --- .../catalog/tabledesc/table_desc_builder.go | 14 +++--- pkg/sql/catalog/tabledesc/validate.go | 7 +-- pkg/sql/catalog/tabledesc/validate_test.go | 4 +- .../logictest/testdata/logic_test/alter_table | 48 +++++++++++++++++++ 4 files changed, 61 insertions(+), 12 deletions(-) diff --git a/pkg/sql/catalog/tabledesc/table_desc_builder.go b/pkg/sql/catalog/tabledesc/table_desc_builder.go index f608a5e0ae41..0bfbf7b80273 100644 --- a/pkg/sql/catalog/tabledesc/table_desc_builder.go +++ b/pkg/sql/catalog/tabledesc/table_desc_builder.go @@ -1313,15 +1313,15 @@ func maybeFixUsesSequencesIDForIdentityColumns( if len(column.UsesSequenceIds) == 1 { continue } - // The first ownership will point to the owner from CREATE TABLE. - tableDesc = builder.getOrInitModifiedDesc() - column = &tableDesc.Columns[idx] if len(column.OwnsSequenceIds) == 0 { - return false, errors.AssertionFailedf("identity column %s (%d) on table %s must own a sequence", - column.Name, - column.ID, - tableDesc.Name) + // With serial_normalization=rowid, a table definition like + // `CREATE TABLE t (a SERIAL GENERATED ALWAYS AS IDENTITY)` + // creates an identity column without any backing sequence. If that's the + // case, there's no need to add a sequence reference. + continue } + tableDesc = builder.getOrInitModifiedDesc() + column = &tableDesc.Columns[idx] column.UsesSequenceIds = append(column.UsesSequenceIds, column.OwnsSequenceIds[0]) wasRepaired = true } diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 7a3cf93b8f8c..b6c72a8822e6 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -1225,9 +1225,10 @@ func (desc *wrapper) validateColumns() error { } } - // For generated as identity columns ensure that the column uses sequences. - if column.NumUsesSequences() != 1 { - return errors.Newf("column is identity without sequence references %q", column.GetName()) + // For generated as identity columns ensure that the column uses sequences + // if it is backed by a sequence. + if column.NumOwnsSequences() == 1 && column.NumUsesSequences() != 1 { + return errors.Newf("column %q is GENERATED BY IDENTITY without sequence references", column.GetName()) } } diff --git a/pkg/sql/catalog/tabledesc/validate_test.go b/pkg/sql/catalog/tabledesc/validate_test.go index dda790034dbb..3d13969bab6e 100644 --- a/pkg/sql/catalog/tabledesc/validate_test.go +++ b/pkg/sql/catalog/tabledesc/validate_test.go @@ -3069,14 +3069,14 @@ func TestValidateTableDesc(t *testing.T) { }, } })}, - {err: `column is identity without sequence references "bar"`, + {err: `column "bar" is GENERATED BY IDENTITY without sequence references`, desc: descpb.TableDescriptor{ ID: 2, ParentID: 1, Name: "foo", FormatVersion: descpb.InterleavedFormatVersion, Columns: []descpb.ColumnDescriptor{ - {ID: 1, Name: "bar", GeneratedAsIdentityType: catpb.GeneratedAsIdentityType_GENERATED_ALWAYS}, + {ID: 1, Name: "bar", OwnsSequenceIds: []descpb.ID{5}, GeneratedAsIdentityType: catpb.GeneratedAsIdentityType_GENERATED_ALWAYS}, }, NextColumnID: 2, }}, diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index f64dc81937eb..fd49fc5d0adc 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -3842,6 +3842,54 @@ CREATE TABLE t_identity_drop_dependency(id INT PRIMARY KEY DEFAULT nextval('publ statement error pgcode 2BP01 pq: cannot drop table t_identity_drop because other objects depend on it ALTER TABLE t_identity_drop ALTER COLUMN b DROP IDENTITY; +let $old_serial_normalization +SHOW serial_normalization + +statement ok +SET serial_normalization = rowid + +# Allow creating an IDENTITY column that isn't backed by a sequence. +statement ok +CREATE TABLE t_serial_identity_no_sequence(a SERIAL PRIMARY KEY GENERATED ALWAYS AS IDENTITY) + +query T +SELECT crdb_internal.pb_to_json('desc', descriptor)->'table'->'columns'->0->>'defaultExpr' +FROM system.descriptor +WHERE id = 't_serial_identity_no_sequence'::regclass::oid +---- +unique_rowid() + +query T +SELECT crdb_internal.pb_to_json('desc', descriptor)->'table'->'columns'->0->>'generatedAsIdentityType' +FROM system.descriptor +WHERE id = 't_serial_identity_no_sequence'::regclass::oid +---- +GENERATED_ALWAYS + +statement ok +SET serial_normalization = sql_sequence + +# Allow creating an IDENTITY column that is backed by a sequence. +statement ok +CREATE TABLE t_serial_identity_with_sequence(a SERIAL PRIMARY KEY GENERATED ALWAYS AS IDENTITY) + +query T +SELECT substring(crdb_internal.pb_to_json('desc', descriptor)->'table'->'columns'->0->>'defaultExpr' FOR 7) +FROM system.descriptor +WHERE id = 't_serial_identity_with_sequence'::regclass::oid +---- +nextval + +query T +SELECT crdb_internal.pb_to_json('desc', descriptor)->'table'->'columns'->0->>'generatedAsIdentityType' +FROM system.descriptor +WHERE id = 't_serial_identity_with_sequence'::regclass::oid +---- +GENERATED_ALWAYS + +statement ok +SET serial_normalization = $old_serial_normalization + subtest comma_syntax statement ok From a6da082a6a3406bc8b421b5ee509ffff18c8e58c Mon Sep 17 00:00:00 2001 From: Stan Rosenberg Date: Tue, 14 Jan 2025 23:30:45 -0500 Subject: [PATCH 118/126] roachprod: user-specified `awsConfigValue` may get overriden by `DefaultConfig` A recent refactoring [1] has exposed a latent bug, wherein user-specified `awsConfigValue`, i.e., `--aws-config` may get overridden by the default configuration. This PR fixes the bug. [1] https://github.com/cockroachdb/cockroach/pull/137394 Epic: none Fixes: #139102 Release note: None --- pkg/roachprod/vm/aws/aws.go | 1 - pkg/roachprod/vm/aws/config.go | 8 +++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/pkg/roachprod/vm/aws/aws.go b/pkg/roachprod/vm/aws/aws.go index b8a139ee56fb..c9945c3595fa 100644 --- a/pkg/roachprod/vm/aws/aws.go +++ b/pkg/roachprod/vm/aws/aws.go @@ -522,7 +522,6 @@ func (o *ProviderOpts) ConfigureClusterFlags(flags *pflag.FlagSet, _ vm.Multiple flags.StringVar(&providerInstance.Profile, ProviderName+"-profile", os.Getenv("AWS_PROFILE"), "Profile to manage cluster in") configFlagVal := awsConfigValue{awsConfig: *DefaultConfig} - providerInstance.Config = &configFlagVal.awsConfig flags.Var(&configFlagVal, ProviderName+"-config", "Path to json for aws configuration, defaults to predefined configuration") } diff --git a/pkg/roachprod/vm/aws/config.go b/pkg/roachprod/vm/aws/config.go index be88120238f8..ee470b49db40 100644 --- a/pkg/roachprod/vm/aws/config.go +++ b/pkg/roachprod/vm/aws/config.go @@ -170,7 +170,13 @@ func (c *awsConfigValue) Set(path string) (err error) { if err != nil { return err } - return json.Unmarshal(data, &c.awsConfig) + err = json.Unmarshal(data, &c.awsConfig) + if err != nil { + return err + } + // Update the provider's config with the user-specified config. + providerInstance.Config = &c.awsConfig + return nil } // Type is part of the pflag.Value interface. From 1d5af8d3ea99a88e0a212a019af987293c8c5bc7 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Wed, 15 Jan 2025 12:33:10 +0000 Subject: [PATCH 119/126] roachprod: fix remote monitor script Previously, roachprod Monitor was updated to use new scripts to be able to detect new processes (See: #136879). But in the event a process dies `systemctl list-units` starts applying formatting to the list entry, for the dead unit. This prevents `awk` from correctly filtering the output. This change adds `--plain` to the `systemctl list-units call` to prevent `systemctl` from adding formatting to a dead system unit. Prior to this change, Monitor was still able to detect the process death due to it not being in the list anymore, but lost the cause of death and exit code. Epic: None Release note: None --- pkg/roachprod/install/scripts/monitor_remote.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/roachprod/install/scripts/monitor_remote.sh b/pkg/roachprod/install/scripts/monitor_remote.sh index 2f76593a3a48..886fb839aba2 100755 --- a/pkg/roachprod/install/scripts/monitor_remote.sh +++ b/pkg/roachprod/install/scripts/monitor_remote.sh @@ -19,7 +19,7 @@ one_shot=#{if .OneShot#}true#{end#} prev_frame="" while :; do # Get all cockroach system units - sysctl_output=$(systemctl list-units cockroach\*.service --type=service --no-legend --no-pager | awk '{print $1}') + sysctl_output=$(systemctl list-units cockroach\*.service --type=service --no-legend --no-pager --plain | awk '{print $1}') frame="" while IFS= read -r name; do # Query the PID and status of the cockroach system unit From 673d36dde4057a0e4527cb1bf33f2e380394b453 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 15 Jan 2025 12:39:03 +0100 Subject: [PATCH 120/126] server: fix clock monotonicity Prior to this PR, the last line in the bash script below would find that the server had crashed with > wall time X is not allowed to be greater than upper bound of Y. The reason is that upon the second start, the node would persist a "temporary" `now()+5s` upper bound but then fail to update it. This PR ensures that the update loop starts unconditionally and maintains (i.e. either updates or erases) the HLC upper bound. This likely fixes other crashes of the same sort when the cluster setting is changed while a node is going down (in which case it's conceivable that a node would boot with a persisted upper bound but without a desire to keep maintaining time timestamp). ``` roachprod wipe local roachprod start local:1 roachprod sql local:1 -- -e "SET CLUSTER SETTING server.clock.persist_upper_bound_interval = '500ms'" sleep 5 roachprod stop local:1 roachprod start local:1 sleep 10 roachprod sql local:1 -- -e "SELECT 1" ``` With the fix, after restarting, the server logs > persisting HLC upper bound is enabled [every 0.50s] a second time (the first time being when the setting is set). Without the fix, we see this log line only once, not to mention the crash. Fixes #137331. Epic: none Release note: None --- pkg/server/clock_monotonicity.go | 18 ++++++++++++------ pkg/server/server_test.go | 2 +- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/pkg/server/clock_monotonicity.go b/pkg/server/clock_monotonicity.go index a8291b6cb79f..59509366fdc9 100644 --- a/pkg/server/clock_monotonicity.go +++ b/pkg/server/clock_monotonicity.go @@ -170,7 +170,6 @@ func periodicallyPersistHLCUpperBound( // persistInterval is the interval used for persisting the // an upper bound of the HLC var persistInterval time.Duration - var ok bool persistHLCUpperBound := func() { if err := clock.RefreshHLCUpperBound( @@ -187,16 +186,19 @@ func periodicallyPersistHLCUpperBound( for { select { - case persistInterval, ok = <-persistHLCUpperBoundIntervalCh: - ticker.Stop() - if !ok { - return + case updatedPersistInterval := <-persistHLCUpperBoundIntervalCh: + if updatedPersistInterval == persistInterval { + // No change. + continue } + persistInterval = updatedPersistInterval + ticker.Stop() if persistInterval > 0 { ticker = tickerFn(persistInterval) persistHLCUpperBound() - log.Ops.Info(context.Background(), "persisting HLC upper bound is enabled") + log.Ops.Infof(context.Background(), "persisting HLC upper bound is enabled [every %.2fs]", + persistInterval.Seconds()) } else { if err := clock.ResetHLCUpperBound(persistHLCUpperBoundFn); err != nil { log.Ops.Fatalf( @@ -242,6 +244,10 @@ func (s *topLevelServer) startPersistingHLCUpperBound( return s.node.SetHLCUpperBound(context.Background(), t) } persistHLCUpperBoundIntervalCh := make(chan time.Duration, 1) + // Seed channel with initial update, then install a callback to update it + // on future changes. SetOnChange does not automatically trigger the callback + // if the setting is initially set. + persistHLCUpperBoundIntervalCh <- persistHLCUpperBoundInterval.Get(&s.st.SV) persistHLCUpperBoundInterval.SetOnChange(&s.st.SV, func(context.Context) { persistHLCUpperBoundIntervalCh <- persistHLCUpperBoundInterval.Get(&s.st.SV) }) diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 9fb32b7bbd6a..ceecf3a50416 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -577,7 +577,7 @@ func TestPersistHLCUpperBound(t *testing.T) { tickProcessedCh := make(chan struct{}) persistHLCUpperBoundIntervalCh := make(chan time.Duration, 1) stopCh := make(chan struct{}, 1) - defer close(persistHLCUpperBoundIntervalCh) + defer close(stopCh) go periodicallyPersistHLCUpperBound( c, From 9ced8fee8e9d6a9a15870f24a22f54866dd26443 Mon Sep 17 00:00:00 2001 From: Shubham Dhama Date: Tue, 1 Oct 2024 09:07:27 +0000 Subject: [PATCH 121/126] multitenant: deduplicate tenant capability check. --- .../authorizer.go | 151 +++++------------- .../testdata/authorizer_enabled | 2 +- 2 files changed, 38 insertions(+), 115 deletions(-) diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go index c169b1e41296..dd14bd35f94d 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go @@ -192,14 +192,6 @@ func newTenantDoesNotHaveCapabilityError(cap tenantcapabilities.ID, req kvpb.Req return errors.Newf("client tenant does not have capability %q (%T)", cap, req) } -var ( - errCannotQueryMetadata = errors.New("client tenant does not have capability to query cluster node metadata") - errCannotQueryTSDB = errors.New("client tenant does not have capability to query timeseries data") - errCannotQueryAllMetrics = errors.New("client tenant does not have capability to query non-tenant metrics") - errCannotUseNodelocal = errors.New("client tenant does not have capability to use nodelocal storage") - errCannotDebugProcess = errors.New("client tenant does not have capability to debug the process") -) - // methodCapability associates a KV method with a capability. The capability can // either be static for all instances of the method, or it can be determined // dynamically by a function based on the request's contents. @@ -316,33 +308,25 @@ func (a *Authorizer) BindReader(reader tenantcapabilities.Reader) { a.capabilitiesReader = reader } -func (a *Authorizer) HasNodeStatusCapability(ctx context.Context, tenID roachpb.TenantID) error { - if tenID.IsSystem() { - return nil - } - entry, mode := a.getMode(ctx, tenID) - switch mode { - case authorizerModeOn: - break - case authorizerModeAllowAll: - return nil - case authorizerModeV222: - return errCannotQueryMetadata - default: - err := errors.AssertionFailedf("unknown authorizer mode: %d", mode) - logcrash.ReportOrPanic(ctx, &a.settings.SV, "%v", err) - return err - } +var ( + errCannotQueryMetadata = errors.New("client tenant does not have capability to query cluster node metadata") + errCannotQueryTSDB = errors.New("client tenant does not have capability to query timeseries data") + errCannotQueryAllMetrics = errors.New("client tenant does not have capability to query non-tenant metrics") + errCannotUseNodelocal = errors.New("client tenant does not have capability to use nodelocal storage") + errCannotDebugProcess = errors.New("client tenant does not have capability to debug the process") +) - if !tenantcapabilities.MustGetBoolByID( - entry.TenantCapabilities, tenantcapabilities.CanViewNodeInfo, - ) { - return errCannotQueryMetadata - } - return nil +var insufficientCapErrMap = map[tenantcapabilities.ID]error{ + tenantcapabilities.CanViewNodeInfo: errCannotQueryMetadata, + tenantcapabilities.CanViewTSDBMetrics: errCannotQueryTSDB, + tenantcapabilities.CanUseNodelocalStorage: errCannotUseNodelocal, + tenantcapabilities.CanDebugProcess: errCannotDebugProcess, + tenantcapabilities.CanViewAllMetrics: errCannotQueryAllMetrics, } -func (a *Authorizer) HasTSDBQueryCapability(ctx context.Context, tenID roachpb.TenantID) error { +func (a *Authorizer) hasCapability( + ctx context.Context, tenID roachpb.TenantID, cap tenantcapabilities.ID, +) error { if tenID.IsSystem() { return nil } @@ -354,47 +338,41 @@ func (a *Authorizer) HasTSDBQueryCapability(ctx context.Context, tenID roachpb.T case authorizerModeAllowAll: return nil case authorizerModeV222: - return errCannotQueryTSDB + return insufficientCapErrMap[cap] default: err := errors.AssertionFailedf("unknown authorizer mode: %d", mode) logcrash.ReportOrPanic(ctx, &a.settings.SV, "%v", err) return err } - if !tenantcapabilities.MustGetBoolByID( - entry.TenantCapabilities, tenantcapabilities.CanViewTSDBMetrics, - ) { - return errCannotQueryTSDB + if !tenantcapabilities.MustGetBoolByID(entry.TenantCapabilities, cap) { + return insufficientCapErrMap[cap] } return nil } +func (a *Authorizer) HasNodeStatusCapability(ctx context.Context, tenID roachpb.TenantID) error { + return a.hasCapability(ctx, tenID, tenantcapabilities.CanViewNodeInfo) +} + +func (a *Authorizer) HasTSDBQueryCapability(ctx context.Context, tenID roachpb.TenantID) error { + return a.hasCapability(ctx, tenID, tenantcapabilities.CanViewTSDBMetrics) +} + func (a *Authorizer) HasNodelocalStorageCapability( ctx context.Context, tenID roachpb.TenantID, ) error { - if tenID.IsSystem() { - return nil - } - entry, mode := a.getMode(ctx, tenID) - switch mode { - case authorizerModeOn: - break - case authorizerModeAllowAll: - return nil - case authorizerModeV222: - return errCannotUseNodelocal - default: - err := errors.AssertionFailedf("unknown authorizer mode: %d", mode) - logcrash.ReportOrPanic(ctx, &a.settings.SV, "%v", err) - return err - } + return a.hasCapability(ctx, tenID, tenantcapabilities.CanUseNodelocalStorage) +} - if !tenantcapabilities.MustGetBoolByID( - entry.TenantCapabilities, tenantcapabilities.CanUseNodelocalStorage, - ) { - return errCannotUseNodelocal - } - return nil +func (a *Authorizer) HasProcessDebugCapability(ctx context.Context, tenID roachpb.TenantID) error { + return a.hasCapability(ctx, tenID, tenantcapabilities.CanDebugProcess) +} + +func (a *Authorizer) HasTSDBAllMetricsCapability( + ctx context.Context, tenID roachpb.TenantID, +) error { + return a.hasCapability(ctx, tenID, tenantcapabilities.CanViewAllMetrics) } // IsExemptFromRateLimiting returns true if the tenant is not subject to rate limiting. @@ -419,61 +397,6 @@ func (a *Authorizer) IsExemptFromRateLimiting(ctx context.Context, tenID roachpb return tenantcapabilities.MustGetBoolByID(entry.TenantCapabilities, tenantcapabilities.ExemptFromRateLimiting) } -func (a *Authorizer) HasProcessDebugCapability(ctx context.Context, tenID roachpb.TenantID) error { - if tenID.IsSystem() { - return nil - } - entry, mode := a.getMode(ctx, tenID) - switch mode { - case authorizerModeOn: - break - case authorizerModeAllowAll: - return nil - case authorizerModeV222: - return errCannotDebugProcess - default: - err := errors.AssertionFailedf("unknown authorizer mode: %d", mode) - logcrash.ReportOrPanic(ctx, &a.settings.SV, "%v", err) - return err - } - - if !tenantcapabilities.MustGetBoolByID( - entry.TenantCapabilities, tenantcapabilities.CanDebugProcess, - ) { - return errCannotDebugProcess - } - return nil -} - -func (a *Authorizer) HasTSDBAllMetricsCapability( - ctx context.Context, tenID roachpb.TenantID, -) error { - if tenID.IsSystem() { - return nil - } - - entry, mode := a.getMode(ctx, tenID) - switch mode { - case authorizerModeOn: - break - case authorizerModeAllowAll: - return nil - case authorizerModeV222: - return errCannotQueryTSDB - default: - err := errors.AssertionFailedf("unknown authorizer mode: %d", mode) - logcrash.ReportOrPanic(ctx, &a.settings.SV, "%v", err) - return err - } - - if !tenantcapabilities.MustGetBoolByID( - entry.TenantCapabilities, tenantcapabilities.CanViewAllMetrics, - ) { - return errCannotQueryAllMetrics - } - return nil -} - // getMode retrieves the authorization mode. func (a *Authorizer) getMode( ctx context.Context, tid roachpb.TenantID, diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/authorizer_enabled b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/authorizer_enabled index 8d880c8be930..426572f306ac 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/authorizer_enabled +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/testdata/authorizer_enabled @@ -50,7 +50,7 @@ client tenant does not have capability to query timeseries data has-tsdb-all-capability ten=10 ---- -client tenant does not have capability to query timeseries data +client tenant does not have capability to query non-tenant metrics # Disable the capability checks by allowing all requests. set-authorizer-mode value=allow-all From edda49c293dae4a3bd4f8f673c6153dbb5cb898d Mon Sep 17 00:00:00 2001 From: Shubham Dhama Date: Thu, 3 Oct 2024 07:33:35 +0000 Subject: [PATCH 122/126] server: enable `_status/connectivity` for secondary tenants (shared-process). This works well for shared-process mode where inter-node network connectivity for a secondary tenant is the same as for the system tenant. However, in external-process mode, this endpoint won't provide a complete picture of network connectivity since the SQL server may run entirely outside the KV node. We may need to extend this endpoint or create a new one for SQL-SQL servers and SQL server to KV nodes. This work is left for the future, and currently, this endpoint only shows KV-KV nodes network connectivity. As a result, this endpoint isn't ready for external-process mode and should only be enabled for secondary tenants. On the backend, there is nothing enforcing this, which shouldn't be a problem. Fixes: #110024 Epic: #CRDB-38968 Release note: None --- .../mocks/tenant_status_server_generated.go | 15 +++++++++++ pkg/kv/kvclient/kvtenant/connector.go | 11 ++++++++ pkg/rpc/auth_tenant.go | 3 +++ pkg/server/serverpb/status.go | 1 + pkg/server/status.go | 25 +++++++++++++++++++ pkg/server/storage_api/network_test.go | 22 ++++++++++------ 6 files changed, 69 insertions(+), 8 deletions(-) diff --git a/pkg/ccl/changefeedccl/mocks/tenant_status_server_generated.go b/pkg/ccl/changefeedccl/mocks/tenant_status_server_generated.go index 7dab92e5a3ee..255b41667509 100644 --- a/pkg/ccl/changefeedccl/mocks/tenant_status_server_generated.go +++ b/pkg/ccl/changefeedccl/mocks/tenant_status_server_generated.go @@ -66,6 +66,21 @@ func (mr *MockTenantStatusServerMockRecorder) HotRangesV2(arg0, arg1 interface{} return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HotRangesV2", reflect.TypeOf((*MockTenantStatusServer)(nil).HotRangesV2), arg0, arg1) } +// NetworkConnectivity mocks base method. +func (m *MockTenantStatusServer) NetworkConnectivity(arg0 context.Context, arg1 *serverpb.NetworkConnectivityRequest) (*serverpb.NetworkConnectivityResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "NetworkConnectivity", arg0, arg1) + ret0, _ := ret[0].(*serverpb.NetworkConnectivityResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// NetworkConnectivity indicates an expected call of NetworkConnectivity. +func (mr *MockTenantStatusServerMockRecorder) NetworkConnectivity(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NetworkConnectivity", reflect.TypeOf((*MockTenantStatusServer)(nil).NetworkConnectivity), arg0, arg1) +} + // Nodes mocks base method. func (m *MockTenantStatusServer) Nodes(arg0 context.Context, arg1 *serverpb.NodesRequest) (*serverpb.NodesResponse, error) { m.ctrl.T.Helper() diff --git a/pkg/kv/kvclient/kvtenant/connector.go b/pkg/kv/kvclient/kvtenant/connector.go index d1b1f2e5ec75..1c3dc6cf5028 100644 --- a/pkg/kv/kvclient/kvtenant/connector.go +++ b/pkg/kv/kvclient/kvtenant/connector.go @@ -684,6 +684,17 @@ func (c *connector) TenantRanges( return } +// NetworkConnectivity implements the serverpb.TenantStatusServer interface +func (c *connector) NetworkConnectivity( + ctx context.Context, req *serverpb.NetworkConnectivityRequest, +) (resp *serverpb.NetworkConnectivityResponse, retErr error) { + retErr = c.withClient(ctx, func(ctx context.Context, client *client) (err error) { + resp, err = client.NetworkConnectivity(ctx, req) + return + }) + return +} + // NewIterator implements the rangedesc.IteratorFactory interface. func (c *connector) NewIterator( ctx context.Context, span roachpb.Span, diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index d923cf643fa4..17da3ab44902 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -118,6 +118,9 @@ func (a tenantAuthorizer) authorize( case "/cockroach.server.serverpb.Status/Ranges": return a.authRanges(tenID) + case "/cockroach.server.serverpb.Status/NetworkConnectivity": + return a.capabilitiesAuthorizer.HasProcessDebugCapability(ctx, tenID) + case "/cockroach.server.serverpb.Status/TransactionContentionEvents": return a.authTenant(tenID) diff --git a/pkg/server/serverpb/status.go b/pkg/server/serverpb/status.go index 9ef5658550f2..f814d1ac3a14 100644 --- a/pkg/server/serverpb/status.go +++ b/pkg/server/serverpb/status.go @@ -90,6 +90,7 @@ type TenantStatusServer interface { // download remote files. A method that mutates state should not be on the // status server and so in the long run we should move it. DownloadSpan(ctx context.Context, request *DownloadSpanRequest) (*DownloadSpanResponse, error) + NetworkConnectivity(context.Context, *NetworkConnectivityRequest) (*NetworkConnectivityResponse, error) } // OptionalNodesStatusServer returns the wrapped NodesStatusServer, if it is diff --git a/pkg/server/status.go b/pkg/server/status.go index 7d3f5678c47d..f24def04fa63 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -2119,6 +2119,31 @@ func (s *statusServer) NodeUI( return &resp, nil } +// NetworkConnectivity collects info about connection statuses across all nodes. +// This isn't tenant-specific information; it's about networking activity across +// all tenants between nodes. It's accessible via the system tenant, and here +// made available to secondary tenants with the `can_debug_process` capability. +// This works well for shared-process mode, but in external-process mode, this +// endpoint won't give a complete picture of network connectivity since the SQL +// server might run entirely outside the KV node. We might need to extend this +// endpoint or create a new one for SQL-SQL servers and SQL server to KV nodes. +// This work is for the future. Currently, this endpoint only shows KV-KV nodes +// network connectivity. So, it's not ready for external-process mode and should +// only be enabled for shared-process mode. There's nothing enforcing this, but +// it shouldn't be a problem. See issue #138156 +func (t *statusServer) NetworkConnectivity( + ctx context.Context, req *serverpb.NetworkConnectivityRequest, +) (*serverpb.NetworkConnectivityResponse, error) { + ctx = t.AnnotateCtx(ctx) + + err := t.privilegeChecker.RequireViewClusterMetadataPermission(ctx) + if err != nil { + return nil, err + } + + return t.sqlServer.tenantConnect.NetworkConnectivity(ctx, req) +} + // NetworkConnectivity collects information about connections statuses across all nodes. func (s *systemStatusServer) NetworkConnectivity( ctx context.Context, req *serverpb.NetworkConnectivityRequest, diff --git a/pkg/server/storage_api/network_test.go b/pkg/server/storage_api/network_test.go index 168a260c5064..fac6388d9c98 100644 --- a/pkg/server/storage_api/network_test.go +++ b/pkg/server/storage_api/network_test.go @@ -11,6 +11,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/server/srvtestutils" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -26,20 +27,25 @@ func TestNetworkConnectivity(t *testing.T) { defer log.Scope(t).Close(t) numNodes := 3 testCluster := serverutils.StartCluster(t, numNodes, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(110024), - }, - ReplicationMode: base.ReplicationManual, }) ctx := context.Background() defer testCluster.Stopper().Stop(ctx) - // TODO(#110024): grant the appropriate capability to the test - // tenant before the connectivity endpoint can be accessed. See - // example in `TestNodeStatusResponse`. - s0 := testCluster.Server(0) + + if s0.TenantController().StartedDefaultTestTenant() { + _, err := s0.SystemLayer().SQLConn(t).Exec( + `ALTER TENANT [$1] GRANT CAPABILITY can_debug_process=true`, + serverutils.TestTenantID().ToUint64(), + ) + require.NoError(t, err) + + serverutils.WaitForTenantCapabilities(t, s0, serverutils.TestTenantID(), map[tenantcapabilities.ID]string{ + tenantcapabilities.CanDebugProcess: "true", + }, "") + } + ts := s0.ApplicationLayer() var resp serverpb.NetworkConnectivityResponse From 9e3c54dfba9db6e7ca7ae9fa5900ece2530d6a2b Mon Sep 17 00:00:00 2001 From: Shubham Dhama Date: Wed, 15 Jan 2025 17:09:35 +0530 Subject: [PATCH 123/126] server: test tenant capabilities for `_status/connectivity` --- pkg/server/storage_api/network_test.go | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/pkg/server/storage_api/network_test.go b/pkg/server/storage_api/network_test.go index fac6388d9c98..72dafda571c2 100644 --- a/pkg/server/storage_api/network_test.go +++ b/pkg/server/storage_api/network_test.go @@ -81,3 +81,25 @@ func TestNetworkConnectivity(t *testing.T) { return nil }) } + +func TestNetworkConnectivityTenantCapability(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + numNodes := 3 + testCluster := serverutils.StartCluster(t, numNodes, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + // Note: We're only testing external-process mode because shared service + // mode tenants have all capabilities. See PR #119211 for more info. + DefaultTestTenant: base.ExternalTestTenantAlwaysEnabled, + }, + ReplicationMode: base.ReplicationManual, + }) + ctx := context.Background() + defer testCluster.Stopper().Stop(ctx) + + var resp serverpb.NetworkConnectivityResponse + err := srvtestutils.GetStatusJSONProto( + testCluster.Server(0).ApplicationLayer(), "connectivity", &resp) + require.ErrorContains(t, err, + "client tenant does not have capability to debug the process") +} From 74c8287f2c809c2f5ee1c42592778a1153733be8 Mon Sep 17 00:00:00 2001 From: Jeff Swenson Date: Mon, 13 Jan 2025 13:48:48 -0500 Subject: [PATCH 124/126] backup: remove support for kv.bulkio.write_metadata_sst.enabled kv.bulkio.write_metadata_sst.enabled causes backup to write out a 'metadata.sst' file. This file was never used in production, but some of the code is used by the new thin manifest format. The code that is still used was moved into new files to fix the build as a refactoring trick to ensure all unused code was deleted. Epic: none Release Note: none --- docs/generated/sql/bnf/stmt_block.bnf | 3 - pkg/backup/BUILD.bazel | 1 - pkg/backup/backup_job.go | 15 - pkg/backup/backupinfo/BUILD.bazel | 16 +- pkg/backup/backupinfo/backup_metadata.go | 1597 ----------------- pkg/backup/backupinfo/backup_metadata_test.go | 343 ---- pkg/backup/backupinfo/desc_sst.go | 342 ++++ pkg/backup/backupinfo/external_sst_util.go | 169 ++ pkg/backup/backupinfo/file_sst.go | 176 ++ pkg/backup/backupinfo/manifest_handling.go | 50 - .../full_cluster_backup_restore_test.go | 6 - pkg/backup/show.go | 67 +- pkg/backup/tenant_backup_nemesis_test.go | 3 - .../in-progress-import-rollback | 8 - pkg/backup/testdata/backup-restore/metadata | 4 - pkg/settings/registry.go | 1 + pkg/sql/parser/sql.y | 8 +- pkg/sql/parser/testdata/show | 12 +- pkg/sql/sem/tree/show.go | 11 - 19 files changed, 703 insertions(+), 2129 deletions(-) delete mode 100644 pkg/backup/backupinfo/backup_metadata.go delete mode 100644 pkg/backup/backupinfo/backup_metadata_test.go create mode 100644 pkg/backup/backupinfo/desc_sst.go create mode 100644 pkg/backup/backupinfo/external_sst_util.go create mode 100644 pkg/backup/backupinfo/file_sst.go diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index c316a7a9f241..743f0089da7c 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1149,7 +1149,6 @@ unreserved_keyword ::= | 'DAY' | 'DEALLOCATE' | 'DEBUG_IDS' - | 'DEBUG_DUMP_METADATA_SST' | 'DECLARE' | 'DELETE' | 'DEFAULTS' @@ -3506,7 +3505,6 @@ show_backup_options ::= | 'ENCRYPTION_PASSPHRASE' '=' string_or_placeholder | 'PRIVILEGES' | 'ENCRYPTION_INFO_DIR' '=' string_or_placeholder - | 'DEBUG_DUMP_METADATA_SST' schema_wildcard ::= wildcard_pattern @@ -3873,7 +3871,6 @@ bare_label_keywords ::= | 'DATABASE' | 'DATABASES' | 'DEALLOCATE' - | 'DEBUG_DUMP_METADATA_SST' | 'DEBUG_IDS' | 'DEC' | 'DECIMAL' diff --git a/pkg/backup/BUILD.bazel b/pkg/backup/BUILD.bazel index 0a1f25b5f753..b87d3093564d 100644 --- a/pkg/backup/BUILD.bazel +++ b/pkg/backup/BUILD.bazel @@ -133,7 +133,6 @@ go_library( "//pkg/util/humanizeutil", "//pkg/util/interval", "//pkg/util/iterutil", - "//pkg/util/json", "//pkg/util/log", "//pkg/util/log/eventpb", "//pkg/util/log/logutil", diff --git a/pkg/backup/backup_job.go b/pkg/backup/backup_job.go index 636b66cf44e5..6c8eef48fade 100644 --- a/pkg/backup/backup_job.go +++ b/pkg/backup/backup_job.go @@ -463,10 +463,6 @@ func backup( // Write a `BACKUP_METADATA` file along with SSTs for all the alloc heavy // fields elided from the `BACKUP_MANIFEST`. - // - // TODO(adityamaru,rhu713): Once backup/restore switches from writing and - // reading backup manifests to `metadata.sst` we can stop writing the slim - // manifest. if backupinfo.WriteMetadataWithExternalSSTsEnabled.Get(&settings.SV) { if err := backupinfo.WriteMetadataWithExternalSSTs(ctx, defaultStore, encryption, &kmsEnv, backupManifest); err != nil { @@ -479,17 +475,6 @@ func backup( return roachpb.RowCount{}, 0, err } - if backupinfo.WriteMetadataSST.Get(&settings.SV) { - if err := backupinfo.WriteBackupMetadataSST(ctx, defaultStore, encryption, &kmsEnv, backupManifest, - statsTable.Statistics); err != nil { - err = errors.Wrap(err, "writing forward-compat metadata sst") - if !build.IsRelease() { - return roachpb.RowCount{}, 0, err - } - log.Warningf(ctx, "%+v", err) - } - } - return backupManifest.EntryCounts, numBackupInstances, nil } diff --git a/pkg/backup/backupinfo/BUILD.bazel b/pkg/backup/backupinfo/BUILD.bazel index 554da257abe2..dc8ab4bb90be 100644 --- a/pkg/backup/backupinfo/BUILD.bazel +++ b/pkg/backup/backupinfo/BUILD.bazel @@ -4,7 +4,9 @@ load("//pkg/testutils:buildutil/buildutil.bzl", "disallowed_imports_test") go_library( name = "backupinfo", srcs = [ - "backup_metadata.go", + "desc_sst.go", + "external_sst_util.go", + "file_sst.go", "manifest_handling.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/backup/backupinfo", @@ -20,7 +22,6 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv/kvpb", - "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/security/username", "//pkg/settings", @@ -34,7 +35,6 @@ go_library( "//pkg/sql/catalog/typedesc", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", - "//pkg/sql/protoreflect", "//pkg/sql/sem/tree", "//pkg/sql/stats", "//pkg/storage", @@ -43,7 +43,6 @@ go_library( "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/ioctx", - "//pkg/util/json", "//pkg/util/log", "//pkg/util/metamorphic", "//pkg/util/mon", @@ -59,16 +58,12 @@ go_library( go_test( name = "backupinfo_test", srcs = [ - "backup_metadata_test.go", "main_test.go", "manifest_handling_test.go", ], deps = [ ":backupinfo", - "//pkg/backup/backupbase", - "//pkg/backup/backupencryption", "//pkg/backup/backuppb", - "//pkg/backup/backuptestutils", "//pkg/base", "//pkg/blobs", "//pkg/ccl", @@ -81,19 +76,14 @@ go_test( "//pkg/security/securitytest", "//pkg/security/username", "//pkg/server", - "//pkg/sql", "//pkg/sql/catalog/descpb", "//pkg/sql/isql", "//pkg/testutils/serverutils", - "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/bulk", "//pkg/util/hlc", - "//pkg/util/ioctx", "//pkg/util/leaktest", "//pkg/util/log", - "//pkg/util/mon", - "//pkg/util/protoutil", "//pkg/util/randutil", "@com_github_stretchr_testify//require", ], diff --git a/pkg/backup/backupinfo/backup_metadata.go b/pkg/backup/backupinfo/backup_metadata.go deleted file mode 100644 index 6c85843540fa..000000000000 --- a/pkg/backup/backupinfo/backup_metadata.go +++ /dev/null @@ -1,1597 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -package backupinfo - -import ( - "bytes" - "context" - "fmt" - "io" - "sort" - "strings" - - "github.com/cockroachdb/cockroach/pkg/backup/backupencryption" - "github.com/cockroachdb/cockroach/pkg/backup/backuppb" - "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" - "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvpb" - "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" - "github.com/cockroachdb/cockroach/pkg/sql/stats" - "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/bulk" - "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/json" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/cockroachdb/errors" -) - -const ( - // MetadataSSTName is the name of the SST file containing the backup metadata. - MetadataSSTName = "metadata.sst" - // BackupMetadataFilesListPath is the name of the SST file containing the - // BackupManifest_Files of the backup. This file is always written in - // conjunction with the `BACKUP_METADATA`. - BackupMetadataFilesListPath = "filelist.sst" - // BackupMetadataDescriptorsListPath is the name of the SST file containing - // the BackupManifest_Descriptors or BackupManifest_DescriptorRevisions of the - // backup. This file is always written in conjunction with the - // `BACKUP_METADATA`. - BackupMetadataDescriptorsListPath = "descriptorslist.sst" - // FileInfoPath is the name of the SST file containing the - // BackupManifest_Files of the backup. - FileInfoPath = "fileinfo.sst" - sstBackupKey = "backup" - sstDescsPrefix = "desc/" - sstFilesPrefix = "file/" - sstNamesPrefix = "name/" - sstSpansPrefix = "span/" - sstStatsPrefix = "stats/" - sstTenantsPrefix = "tenant/" -) - -var iterOpts = storage.IterOptions{ - KeyTypes: storage.IterKeyTypePointsOnly, - LowerBound: keys.LocalMax, - UpperBound: keys.MaxKey, -} - -// WriteFilesListSST is responsible for constructing and writing the -// filePathInfo to dest. This file contains the `BackupManifest_Files` of the -// backup. -func WriteFilesListSST( - ctx context.Context, - dest cloud.ExternalStorage, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, - manifest *backuppb.BackupManifest, - filePathInfo string, -) error { - return writeFilesSST(ctx, manifest, dest, enc, kmsEnv, filePathInfo) -} - -// WriteBackupMetadataSST is responsible for constructing and writing the -// `metadata.sst` to dest. This file contains the metadata corresponding to this -// backup. -func WriteBackupMetadataSST( - ctx context.Context, - dest cloud.ExternalStorage, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, - manifest *backuppb.BackupManifest, - stats []*stats.TableStatisticProto, -) error { - var w io.WriteCloser - ctx, cancel := context.WithCancel(ctx) - defer func() { - cancel() // cancel before Close() to abort write on err returns. - if w != nil { - w.Close() - } - }() - - w, err := makeWriter(ctx, dest, MetadataSSTName, enc, kmsEnv) - if err != nil { - return err - } - - if err := constructMetadataSST(ctx, dest, enc, kmsEnv, w, manifest, stats); err != nil { - return err - } - - // Explicitly close to flush and check for errors do so before defer's cancel - // which would abort. Then nil out w to avoid defer double-closing. - err = w.Close() - w = nil - return err -} - -func makeWriter( - ctx context.Context, - dest cloud.ExternalStorage, - filename string, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, -) (io.WriteCloser, error) { - w, err := dest.Writer(ctx, filename) - if err != nil { - return nil, err - } - - if enc != nil { - key, err := backupencryption.GetEncryptionKey(ctx, enc, kmsEnv) - if err != nil { - return nil, err - } - encW, err := storageccl.EncryptingWriter(w, key) - if err != nil { - return nil, err - } - w = encW - } - return w, nil -} - -func constructMetadataSST( - ctx context.Context, - dest cloud.ExternalStorage, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, - w io.Writer, - m *backuppb.BackupManifest, - stats []*stats.TableStatisticProto, -) error { - // TODO(dt): use a seek-optimized SST writer instead. - sst := storage.MakeTransportSSTWriter(ctx, dest.Settings(), w) - defer sst.Close() - - // The following steps must be done in-order, by key prefix. - - if err := writeManifestToMetadata(ctx, sst, m); err != nil { - return err - } - - if err := writeDescsToMetadata(ctx, sst, m); err != nil { - return err - } - - if err := writeFilesToMetadata(ctx, sst, m, dest, enc, kmsEnv, FileInfoPath); err != nil { - return err - } - - if err := writeNamesToMetadata(ctx, sst, m); err != nil { - return err - } - - if err := writeSpansToMetadata(ctx, sst, m); err != nil { - return err - } - - if err := writeStatsToMetadata(ctx, sst, stats); err != nil { - return err - } - - if err := writeTenantsToMetadata(ctx, sst, m); err != nil { - return err - } - - return sst.Finish() -} - -func writeManifestToMetadata( - ctx context.Context, sst storage.SSTWriter, m *backuppb.BackupManifest, -) error { - info := *m - info.Descriptors = nil - info.DescriptorChanges = nil - info.Files = nil - info.Spans = nil - info.StatisticsFilenames = nil - info.IntroducedSpans = nil - info.Tenants = nil - - b, err := protoutil.Marshal(&info) - if err != nil { - return err - } - return sst.PutUnversioned(roachpb.Key(sstBackupKey), b) -} - -func DescChangesLess( - left *backuppb.BackupManifest_DescriptorRevision, - right *backuppb.BackupManifest_DescriptorRevision, -) bool { - if left.ID != right.ID { - return left.ID < right.ID - } - - return !left.Time.Less(right.Time) -} - -func writeDescsToMetadata( - ctx context.Context, sst storage.SSTWriter, m *backuppb.BackupManifest, -) error { - // Add descriptors from revisions if available, Descriptors if not. - if len(m.DescriptorChanges) > 0 { - sort.Slice(m.DescriptorChanges, func(i, j int) bool { - return DescChangesLess(&m.DescriptorChanges[i], &m.DescriptorChanges[j]) - }) - for _, i := range m.DescriptorChanges { - k := encodeDescSSTKey(i.ID) - var b []byte - if i.Desc != nil { - t, _, _, _, _ := descpb.GetDescriptors(i.Desc) - if t == nil || !t.Dropped() { - bytes, err := protoutil.Marshal(i.Desc) - if err != nil { - return err - } - b = bytes - } - } - if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: i.Time}, b); err != nil { - return err - } - - } - } else { - sort.Slice(m.Descriptors, func(i, j int) bool { - return descID(m.Descriptors[i]) < descID(m.Descriptors[j]) - }) - for _, i := range m.Descriptors { - id := descID(i) - k := encodeDescSSTKey(id) - b, err := protoutil.Marshal(&i) - if err != nil { - return err - } - - // Put descriptors at start time. For non-rev backups this timestamp - // doesn't matter. For the special case where there were no descriptor - // changes in an incremental backup, it's helpful to have existing - // descriptors at the start time, so we don't have to look back further - // than the very last backup. - if m.StartTime.IsEmpty() || m.MVCCFilter == backuppb.MVCCFilter_Latest { - if err := sst.PutUnversioned(k, b); err != nil { - return err - } - } else { - if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: m.StartTime}, b); err != nil { - return err - } - } - } - } - return nil -} - -// WriteDescsSST is responsible for writing the SST containing the Descriptor -// and DescriptorChanges field of the input BackupManifest. If DescriptorChanges -// is non-empty, then the descriptor changes will be written to the SST with the -// MVCC timestamp equal to the revision time. Otherwise, contents of the -// Descriptors field will be written to the SST with an empty MVCC timestamp. -func WriteDescsSST( - ctx context.Context, - m *backuppb.BackupManifest, - dest cloud.ExternalStorage, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, - path string, -) error { - w, err := makeWriter(ctx, dest, path, enc, kmsEnv) - if err != nil { - return err - } - defer w.Close() - descSST := storage.MakeTransportSSTWriter(ctx, dest.Settings(), w) - defer descSST.Close() - - if err := writeDescsToMetadata(ctx, descSST, m); err != nil { - return err - } - - if err := descSST.Finish(); err != nil { - return err - } - - return w.Close() -} - -// FileCmp gives an ordering to two backuppb.BackupManifest_File. -func FileCmp(left backuppb.BackupManifest_File, right backuppb.BackupManifest_File) int { - if cmp := left.Span.Key.Compare(right.Span.Key); cmp != 0 { - return cmp - } - - return strings.Compare(left.Path, right.Path) -} - -func writeFilesSST( - ctx context.Context, - m *backuppb.BackupManifest, - dest cloud.ExternalStorage, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, - fileInfoPath string, -) error { - w, err := makeWriter(ctx, dest, fileInfoPath, enc, kmsEnv) - if err != nil { - return err - } - defer w.Close() - fileSST := storage.MakeTransportSSTWriter(ctx, dest.Settings(), w) - defer fileSST.Close() - - // Sort and write all of the files into a single file info SST. - sort.Slice(m.Files, func(i, j int) bool { - return FileCmp(m.Files[i], m.Files[j]) < 0 - }) - - for i := range m.Files { - file := m.Files[i] - b, err := protoutil.Marshal(&file) - if err != nil { - return err - } - if err := fileSST.PutUnversioned(encodeFileSSTKey(file.Span.Key, file.Path), b); err != nil { - return err - } - } - - err = fileSST.Finish() - if err != nil { - return err - } - return w.Close() -} - -func writeFilesToMetadata( - ctx context.Context, - sst storage.SSTWriter, - m *backuppb.BackupManifest, - dest cloud.ExternalStorage, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, - fileInfoPath string, -) error { - if err := writeFilesSST(ctx, m, dest, enc, kmsEnv, fileInfoPath); err != nil { - return err - } - // Write the file info into the main metadata SST. - return sst.PutUnversioned(encodeFilenameSSTKey(fileInfoPath), nil) -} - -type name struct { - parent, parentSchema descpb.ID - name string - id descpb.ID - ts hlc.Timestamp -} - -type namespace []name - -func (a namespace) Len() int { return len(a) } -func (a namespace) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -func (a namespace) Less(i, j int) bool { - if a[i].parent == a[j].parent { - if a[i].parentSchema == a[j].parentSchema { - cmpName := strings.Compare(a[i].name, a[j].name) - if cmpName == 0 { - cmpTimestamp := a[j].ts.Compare(a[i].ts) - if cmpTimestamp == 0 { - return a[i].id > a[j].id - } - if a[i].ts.IsEmpty() { - return true - } - return cmpTimestamp < 0 - } - return cmpName < 0 - } - return a[i].parentSchema < a[j].parentSchema - } - return a[i].parent < a[j].parent -} - -func writeNamesToMetadata( - ctx context.Context, sst storage.SSTWriter, m *backuppb.BackupManifest, -) error { - revs := m.DescriptorChanges - if len(revs) == 0 { - revs = make([]backuppb.BackupManifest_DescriptorRevision, len(m.Descriptors)) - for i := range m.Descriptors { - revs[i].Desc = &m.Descriptors[i] - revs[i].Time = m.EndTime - revs[i].ID = descID(m.Descriptors[i]) - } - } - - names := make(namespace, len(revs)) - - for i, rev := range revs { - names[i].id = rev.ID - names[i].ts = rev.Time - tb, db, typ, sc, fn := descpb.GetDescriptors(rev.Desc) - if db != nil { - names[i].name = db.Name - if db.State == descpb.DescriptorState_DROP { - names[i].id = 0 - } - } else if sc != nil { - names[i].name = sc.Name - names[i].parent = sc.ParentID - if sc.State == descpb.DescriptorState_DROP { - names[i].id = 0 - } - } else if tb != nil { - names[i].name = tb.Name - names[i].parent = tb.ParentID - names[i].parentSchema = keys.PublicSchemaID - if s := tb.UnexposedParentSchemaID; s != descpb.InvalidID { - names[i].parentSchema = s - } - if tb.State == descpb.DescriptorState_DROP { - names[i].id = 0 - } - } else if typ != nil { - names[i].name = typ.Name - names[i].parent = typ.ParentID - names[i].parentSchema = typ.ParentSchemaID - if typ.State == descpb.DescriptorState_DROP { - names[i].id = 0 - } - } else if fn != nil { - names[i].name = fn.Name - names[i].parent = fn.ParentID - names[i].parentSchema = fn.ParentSchemaID - if fn.State == descpb.DescriptorState_DROP { - names[i].id = 0 - } - } - } - sort.Sort(names) - - for i, rev := range names { - if rev.name == "" { - continue - } - if i > 0 { - prev := names[i-1] - prev.id = rev.id - if prev == rev { - // Name has multiple ID mappings at the same timestamp. - // At most one of these can be non-zero. Zero IDs correspond to name - // entry deletions following a DROP. If there is a non-zero ID, it means - // that the DROP was followed by a RENAME or a CREATE using the same - // name. - // The sort ordering of the names guarantees that the zero IDs are last, - // we therefore keep only the first mapping. - if rev.id != 0 { - return errors.AssertionFailedf( - "attempting to write duplicate name mappings for key (%d, %d, %q) at %s: IDs %d and %d", - rev.parent, rev.parentSchema, rev.name, rev.ts, prev.id, rev.id) - } - continue - } - prev = names[i-1] - prev.ts = rev.ts - if prev == rev { - // Ignore identical mappings at subsequent timestamps. - continue - } - } - k := encodeNameSSTKey(rev.parent, rev.parentSchema, rev.name) - v := encoding.EncodeUvarintAscending(nil, uint64(rev.id)) - if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: rev.ts}, v); err != nil { - return err - } - } - - return nil -} - -func writeSpansToMetadata( - ctx context.Context, sst storage.SSTWriter, m *backuppb.BackupManifest, -) error { - sort.Sort(roachpb.Spans(m.Spans)) - sort.Sort(roachpb.Spans(m.IntroducedSpans)) - - for i, j := 0, 0; i < len(m.Spans) || j < len(m.IntroducedSpans); { - var sp roachpb.Span - var ts hlc.Timestamp - - // Merge spans and introduced spans into one series of spans where the ts on - // each is 0 if it was introduced or the backup start time otherwise. - if j >= len(m.IntroducedSpans) { - sp = m.Spans[i] - ts = m.StartTime - i++ - } else if i >= len(m.Spans) { - sp = m.IntroducedSpans[j] - ts = hlc.Timestamp{} - j++ - } else { - cmp := m.Spans[i].Key.Compare(m.IntroducedSpans[j].Key) - if cmp < 0 { - sp = m.Spans[i] - ts = m.StartTime - i++ - } else { - sp = m.IntroducedSpans[j] - ts = hlc.Timestamp{} - j++ - } - } - if ts.IsEmpty() { - if err := sst.PutUnversioned(encodeSpanSSTKey(sp), nil); err != nil { - return err - } - } else { - k := storage.MVCCKey{Key: encodeSpanSSTKey(sp), Timestamp: ts} - if err := sst.PutRawMVCC(k, nil); err != nil { - return err - } - } - } - return nil -} - -func writeStatsToMetadata( - ctx context.Context, sst storage.SSTWriter, stats []*stats.TableStatisticProto, -) error { - sort.Slice(stats, func(i, j int) bool { - return stats[i].TableID < stats[j].TableID || (stats[i].TableID == stats[j].TableID && stats[i].StatisticID < stats[j].StatisticID) - }) - - for _, s := range stats { - b, err := protoutil.Marshal(s) - if err != nil { - return err - } - if err := sst.PutUnversioned(encodeStatSSTKey(s.TableID, s.StatisticID), b); err != nil { - return err - } - } - return nil -} - -func writeTenantsToMetadata( - ctx context.Context, sst storage.SSTWriter, m *backuppb.BackupManifest, -) error { - sort.Slice(m.Tenants, func(i, j int) bool { return m.Tenants[i].ID < m.Tenants[j].ID }) - for _, i := range m.Tenants { - b, err := protoutil.Marshal(&i) - if err != nil { - return err - } - if err := sst.PutUnversioned(encodeTenantSSTKey(i.ID), b); err != nil { - return err - } - } - return nil -} - -func descID(in descpb.Descriptor) descpb.ID { - switch i := in.Union.(type) { - case *descpb.Descriptor_Table: - return i.Table.ID - case *descpb.Descriptor_Database: - return i.Database.ID - case *descpb.Descriptor_Type: - return i.Type.ID - case *descpb.Descriptor_Schema: - return i.Schema.ID - case *descpb.Descriptor_Function: - return i.Function.ID - default: - panic(fmt.Sprintf("unknown desc %T", in)) - } -} - -func deprefix(key roachpb.Key, prefix string) (roachpb.Key, error) { - if !bytes.HasPrefix(key, []byte(prefix)) { - return nil, errors.Errorf("malformed key missing expected prefix %s: %q", prefix, key) - } - return key[len(prefix):], nil -} - -func encodeDescSSTKey(id descpb.ID) roachpb.Key { - return roachpb.Key(encoding.EncodeUvarintAscending([]byte(sstDescsPrefix), uint64(id))) -} - -func decodeDescSSTKey(key roachpb.Key) (descpb.ID, error) { - key, err := deprefix(key, sstDescsPrefix) - if err != nil { - return 0, err - } - _, id, err := encoding.DecodeUvarintAscending(key) - return descpb.ID(id), err -} - -func encodeFileSSTKey(spanStart roachpb.Key, filename string) roachpb.Key { - buf := make([]byte, 0) - buf = encoding.EncodeBytesAscending(buf, spanStart) - return roachpb.Key(encoding.EncodeStringAscending(buf, filename)) -} - -func encodeFilenameSSTKey(filename string) roachpb.Key { - return encoding.EncodeStringAscending([]byte(sstFilesPrefix), filename) -} - -func decodeUnsafeFileSSTKey(key roachpb.Key) (roachpb.Key, string, error) { - key, spanStart, err := encoding.DecodeBytesAscending(key, nil) - if err != nil { - return nil, "", err - } - _, filename, err := encoding.DecodeUnsafeStringAscending(key, nil) - if err != nil { - return nil, "", err - } - return roachpb.Key(spanStart), filename, err -} - -func decodeUnsafeFileInfoSSTKey(key roachpb.Key) (string, error) { - key, err := deprefix(key, sstFilesPrefix) - if err != nil { - return "", err - } - - _, path, err := encoding.DecodeUnsafeStringAscending(key, nil) - if err != nil { - return "", err - } - return path, err -} - -func encodeNameSSTKey(parentDB, parentSchema descpb.ID, name string) roachpb.Key { - buf := []byte(sstNamesPrefix) - buf = encoding.EncodeUvarintAscending(buf, uint64(parentDB)) - buf = encoding.EncodeUvarintAscending(buf, uint64(parentSchema)) - return roachpb.Key(encoding.EncodeStringAscending(buf, name)) -} - -func decodeUnsafeNameSSTKey(key roachpb.Key) (descpb.ID, descpb.ID, string, error) { - key, err := deprefix(key, sstNamesPrefix) - if err != nil { - return 0, 0, "", err - } - key, parentID, err := encoding.DecodeUvarintAscending(key) - if err != nil { - return 0, 0, "", err - } - key, schemaID, err := encoding.DecodeUvarintAscending(key) - if err != nil { - return 0, 0, "", err - } - _, name, err := encoding.DecodeUnsafeStringAscending(key, nil) - if err != nil { - return 0, 0, "", err - } - return descpb.ID(parentID), descpb.ID(schemaID), name, nil -} - -func encodeSpanSSTKey(span roachpb.Span) roachpb.Key { - buf := encoding.EncodeBytesAscending([]byte(sstSpansPrefix), span.Key) - return roachpb.Key(encoding.EncodeBytesAscending(buf, span.EndKey)) -} - -func decodeSpanSSTKey(key roachpb.Key) (roachpb.Span, error) { - key, err := deprefix(key, sstSpansPrefix) - if err != nil { - return roachpb.Span{}, err - } - key, start, err := encoding.DecodeBytesAscending(key, nil) - if err != nil { - return roachpb.Span{}, err - } - _, end, err := encoding.DecodeBytesAscending(key, nil) - return roachpb.Span{Key: start, EndKey: end}, err -} - -func encodeStatSSTKey(id descpb.ID, statID uint64) roachpb.Key { - buf := encoding.EncodeUvarintAscending([]byte(sstStatsPrefix), uint64(id)) - return roachpb.Key(encoding.EncodeUvarintAscending(buf, statID)) -} - -func decodeStatSSTKey(key roachpb.Key) (descpb.ID, uint64, error) { - key, err := deprefix(key, sstStatsPrefix) - if err != nil { - return 0, 0, err - } - key, id, err := encoding.DecodeUvarintAscending(key) - if err != nil { - return 0, 0, err - } - _, stat, err := encoding.DecodeUvarintAscending(key) - return descpb.ID(id), stat, err -} - -func encodeTenantSSTKey(id uint64) roachpb.Key { - return encoding.EncodeUvarintAscending([]byte(sstTenantsPrefix), id) -} - -func decodeTenantSSTKey(key roachpb.Key) (uint64, error) { - key, err := deprefix(key, sstTenantsPrefix) - if err != nil { - return 0, err - } - _, id, err := encoding.DecodeUvarintAscending(key) - if err != nil { - return 0, err - } - return id, nil -} - -func pbBytesToJSON(in []byte, msg protoutil.Message) (json.JSON, error) { - if err := protoutil.Unmarshal(in, msg); err != nil { - return nil, err - } - j, err := protoreflect.MessageToJSON(msg, protoreflect.FmtFlags{}) - if err != nil { - return nil, err - } - return j, nil -} - -func debugDumpFileSST( - ctx context.Context, - store cloud.ExternalStorage, - fileInfoPath string, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, - out func(rawKey, readableKey string, value json.JSON) error, -) error { - var encOpts *kvpb.FileEncryptionOptions - if enc != nil { - key, err := backupencryption.GetEncryptionKey(ctx, enc, kmsEnv) - if err != nil { - return err - } - encOpts = &kvpb.FileEncryptionOptions{Key: key} - } - iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: store, FilePath: fileInfoPath}}, encOpts, iterOpts) - if err != nil { - return err - } - defer iter.Close() - for iter.SeekGE(storage.MVCCKey{}); ; iter.Next() { - ok, err := iter.Valid() - if err != nil { - return err - } - if !ok { - break - } - k := iter.UnsafeKey() - spanStart, path, err := decodeUnsafeFileSSTKey(k.Key) - if err != nil { - return err - } - v, err := iter.UnsafeValue() - if err != nil { - return err - } - f, err := pbBytesToJSON(v, &backuppb.BackupManifest_File{}) - if err != nil { - return err - } - if err := out(k.String(), fmt.Sprintf("file %s (%s)", path, spanStart.String()), f); err != nil { - return err - } - } - - return nil -} - -// DebugDumpMetadataSST is for debugging a metadata SST. -func DebugDumpMetadataSST( - ctx context.Context, - store cloud.ExternalStorage, - path string, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, - out func(rawKey, readableKey string, value json.JSON) error, -) error { - var encOpts *kvpb.FileEncryptionOptions - if enc != nil { - key, err := backupencryption.GetEncryptionKey(ctx, enc, kmsEnv) - if err != nil { - return err - } - encOpts = &kvpb.FileEncryptionOptions{Key: key} - } - iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: store, - FilePath: path}}, encOpts, iterOpts) - if err != nil { - return err - } - defer iter.Close() - - for iter.SeekGE(storage.MVCCKey{}); ; iter.Next() { - ok, err := iter.Valid() - if err != nil { - return err - } - if !ok { - break - } - k := iter.UnsafeKey() - switch { - case bytes.Equal(k.Key, []byte(sstBackupKey)): - v, err := iter.UnsafeValue() - if err != nil { - return err - } - info, err := pbBytesToJSON(v, &backuppb.BackupManifest{}) - if err != nil { - return err - } - if err := out(k.String(), "backup info", info); err != nil { - return err - } - - case bytes.HasPrefix(k.Key, []byte(sstDescsPrefix)): - id, err := decodeDescSSTKey(k.Key) - if err != nil { - return err - } - var desc json.JSON - if v, err := iter.UnsafeValue(); err == nil && len(v) > 0 { - desc, err = pbBytesToJSON(v, &descpb.Descriptor{}) - if err != nil { - return err - } - } else if err != nil { - return err - } - if err := out(k.String(), fmt.Sprintf("desc %d @ %v", id, k.Timestamp), desc); err != nil { - return err - } - - case bytes.HasPrefix(k.Key, []byte(sstFilesPrefix)): - p, err := decodeUnsafeFileInfoSSTKey(k.Key) - if err != nil { - return err - } - if err := out(k.String(), fmt.Sprintf("file info @ %s", p), nil); err != nil { - return err - } - if err := debugDumpFileSST(ctx, store, p, enc, kmsEnv, out); err != nil { - return err - } - case bytes.HasPrefix(k.Key, []byte(sstNamesPrefix)): - db, sc, name, err := decodeUnsafeNameSSTKey(k.Key) - if err != nil { - return err - } - var id uint64 - if v, err := iter.UnsafeValue(); err == nil && len(v) > 0 { - _, id, err = encoding.DecodeUvarintAscending(v) - if err != nil { - return err - } - } else if err != nil { - return err - } - mapping := fmt.Sprintf("name db %d / schema %d / %q @ %v -> %d", db, sc, name, k.Timestamp, id) - if err := out(k.String(), mapping, nil); err != nil { - return err - } - - case bytes.HasPrefix(k.Key, []byte(sstSpansPrefix)): - span, err := decodeSpanSSTKey(k.Key) - if err != nil { - return err - } - if err := out(k.String(), fmt.Sprintf("span %s @ %v", span, k.Timestamp), nil); err != nil { - return err - } - - case bytes.HasPrefix(k.Key, []byte(sstStatsPrefix)): - tblID, statID, err := decodeStatSSTKey(k.Key) - if err != nil { - return err - } - v, err := iter.UnsafeValue() - if err != nil { - return err - } - s, err := pbBytesToJSON(v, &stats.TableStatisticProto{}) - if err != nil { - return err - } - if err := out(k.String(), fmt.Sprintf("stats tbl %d, id %d", tblID, statID), s); err != nil { - return err - } - - case bytes.HasPrefix(k.Key, []byte(sstTenantsPrefix)): - id, err := decodeTenantSSTKey(k.Key) - if err != nil { - return err - } - v, err := iter.UnsafeValue() - if err != nil { - return err - } - i, err := pbBytesToJSON(v, &mtinfopb.ProtoInfo{}) - if err != nil { - return err - } - if err := out(k.String(), fmt.Sprintf("tenant %d", id), i); err != nil { - return err - } - - default: - v, err := iter.UnsafeValue() - if err != nil { - return err - } - if err := out(k.String(), "unknown", json.FromString(fmt.Sprintf("%q", v))); err != nil { - return err - } - } - } - - return nil -} - -// BackupMetadata holds all of the data in backuppb.BackupManifest except a few repeated -// fields such as descriptors or spans. BackupMetadata provides iterator methods -// so that the excluded fields can be accessed in a streaming manner. -type BackupMetadata struct { - backuppb.BackupManifest - store cloud.ExternalStorage - enc *jobspb.BackupEncryptionOptions - filename string - kmsEnv cloud.KMSEnv -} - -// NewBackupMetadata returns a new BackupMetadata instance. -func NewBackupMetadata( - ctx context.Context, - exportStore cloud.ExternalStorage, - sstFileName string, - encryption *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, -) (*BackupMetadata, error) { - var encOpts *kvpb.FileEncryptionOptions - if encryption != nil { - key, err := backupencryption.GetEncryptionKey(ctx, encryption, kmsEnv) - if err != nil { - return nil, err - } - encOpts = &kvpb.FileEncryptionOptions{Key: key} - } - iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: exportStore, - FilePath: sstFileName}}, encOpts, iterOpts) - if err != nil { - return nil, err - } - defer iter.Close() - - var sstManifest backuppb.BackupManifest - iter.SeekGE(storage.MakeMVCCMetadataKey([]byte(sstBackupKey))) - ok, err := iter.Valid() - if err != nil { - return nil, err - } - if !ok || !iter.UnsafeKey().Key.Equal([]byte(sstBackupKey)) { - return nil, errors.Errorf("metadata SST does not contain backup manifest") - } - - v, err := iter.UnsafeValue() - if err != nil { - return nil, err - } - if err := protoutil.Unmarshal(v, &sstManifest); err != nil { - return nil, err - } - - return &BackupMetadata{BackupManifest: sstManifest, store: exportStore, - enc: encryption, filename: sstFileName, kmsEnv: kmsEnv}, nil -} - -// SpanIterator is a simple iterator to iterate over roachpb.Spans. -type SpanIterator struct { - backing bytesIter - filter func(key storage.MVCCKey) bool - value *roachpb.Span - err error -} - -// NewSpanIter creates a new SpanIterator for the backup metadata. -func (b *BackupMetadata) NewSpanIter(ctx context.Context) bulk.Iterator[roachpb.Span] { - backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstSpansPrefix), b.enc, - true, b.kmsEnv) - it := SpanIterator{ - backing: backing, - } - it.Next() - return &it -} - -// NewIntroducedSpanIter creates a new IntroducedSpanIterator for the backup metadata. -func (b *BackupMetadata) NewIntroducedSpanIter(ctx context.Context) bulk.Iterator[roachpb.Span] { - backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstSpansPrefix), b.enc, - false, b.kmsEnv) - - it := SpanIterator{ - backing: backing, - filter: func(key storage.MVCCKey) bool { - return key.Timestamp == hlc.Timestamp{} - }, - } - it.Next() - return &it -} - -// Close closes the iterator. -func (si *SpanIterator) Close() { - si.backing.close() -} - -func (si *SpanIterator) Valid() (bool, error) { - if si.err != nil { - return false, si.err - } - return si.value != nil, si.err -} - -// Value implements the Iterator interface. -func (si *SpanIterator) Value() roachpb.Span { - if si.value == nil { - return roachpb.Span{} - } - return *si.value -} - -// Next implements the Iterator interface. -func (si *SpanIterator) Next() { - wrapper := resultWrapper{} - var nextSpan *roachpb.Span - - for si.backing.next(&wrapper) { - if si.filter == nil || si.filter(wrapper.key) { - sp, err := decodeSpanSSTKey(wrapper.key.Key) - if err != nil { - si.err = err - return - } - - nextSpan = &sp - break - } - } - - si.value = nextSpan -} - -// FileIterator is a simple iterator to iterate over backuppb.BackupManifest_File. -type FileIterator struct { - mergedIterator storage.SimpleMVCCIterator - err error - file *backuppb.BackupManifest_File -} - -// NewFileIter creates a new FileIterator for the backup metadata. -func (b *BackupMetadata) NewFileIter( - ctx context.Context, -) (bulk.Iterator[*backuppb.BackupManifest_File], error) { - fileInfoIter := makeBytesIter(ctx, b.store, b.filename, []byte(sstFilesPrefix), b.enc, - false, b.kmsEnv) - defer fileInfoIter.close() - - var storeFiles []storageccl.StoreFile - var encOpts *kvpb.FileEncryptionOptions - if b.enc != nil { - key, err := backupencryption.GetEncryptionKey(ctx, b.enc, b.kmsEnv) - if err != nil { - return nil, err - } - encOpts = &kvpb.FileEncryptionOptions{Key: key} - } - - result := resultWrapper{} - for fileInfoIter.next(&result) { - path, err := decodeUnsafeFileInfoSSTKey(result.key.Key) - if err != nil { - break - } - storeFiles = append(storeFiles, storageccl.StoreFile{Store: b.store, - FilePath: path}) - } - - if fileInfoIter.err() != nil { - return nil, fileInfoIter.err() - } - return newFileSSTIter(ctx, storeFiles, encOpts) -} - -// NewFileSSTIter creates a new FileIterator to iterate over the storeFile. -// It is the caller's responsibility to Close() the returned iterator. -func NewFileSSTIter( - ctx context.Context, storeFile storageccl.StoreFile, encOpts *kvpb.FileEncryptionOptions, -) (*FileIterator, error) { - return newFileSSTIter(ctx, []storageccl.StoreFile{storeFile}, encOpts) -} - -func newFileSSTIter( - ctx context.Context, storeFiles []storageccl.StoreFile, encOpts *kvpb.FileEncryptionOptions, -) (*FileIterator, error) { - iter, err := storageccl.ExternalSSTReader(ctx, storeFiles, encOpts, iterOpts) - if err != nil { - return nil, err - } - iter.SeekGE(storage.MVCCKey{}) - fi := &FileIterator{mergedIterator: iter} - fi.Next() - return fi, nil -} - -// Close closes the iterator. -func (fi *FileIterator) Close() { - fi.mergedIterator.Close() -} - -// Valid indicates whether or not the iterator is pointing to a valid value. -func (fi *FileIterator) Valid() (bool, error) { - if fi.err != nil { - return false, fi.err - } - - return fi.file != nil, nil -} - -// Value implements the Iterator interface. -func (fi *FileIterator) Value() *backuppb.BackupManifest_File { - return fi.file -} - -// Next implements the Iterator interface. -func (fi *FileIterator) Next() { - if fi.err != nil { - return - } - - if ok, err := fi.mergedIterator.Valid(); !ok { - if err != nil { - fi.err = err - } - fi.file = nil - return - } - - v, err := fi.mergedIterator.UnsafeValue() - if err != nil { - fi.err = err - return - } - - file := &backuppb.BackupManifest_File{} - err = protoutil.Unmarshal(v, file) - if err != nil { - fi.err = err - return - } - - fi.file = file - fi.mergedIterator.Next() -} - -// DescIterator is a simple iterator to iterate over descpb.Descriptors. -type DescIterator struct { - backing bytesIter - value *descpb.Descriptor - err error -} - -// NewDescIter creates a new DescIterator for the backup metadata. -func (b *BackupMetadata) NewDescIter(ctx context.Context) bulk.Iterator[*descpb.Descriptor] { - backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstDescsPrefix), b.enc, true, b.kmsEnv) - it := DescIterator{ - backing: backing, - } - it.Next() - return &it -} - -// Close closes the iterator. -func (di *DescIterator) Close() { - di.backing.close() -} - -// Valid implements the Iterator interface. -func (di *DescIterator) Valid() (bool, error) { - if di.err != nil { - return false, di.err - } - return di.value != nil, nil -} - -// Value implements the Iterator interface. -func (di *DescIterator) Value() *descpb.Descriptor { - return di.value -} - -// Next implements the Iterator interface. -func (di *DescIterator) Next() { - if di.err != nil { - return - } - - wrapper := resultWrapper{} - var nextValue *descpb.Descriptor - descHolder := descpb.Descriptor{} - for di.backing.next(&wrapper) { - err := protoutil.Unmarshal(wrapper.value, &descHolder) - if err != nil { - di.err = err - return - } - - tbl, db, typ, sc, fn := descpb.GetDescriptors(&descHolder) - if tbl != nil || db != nil || typ != nil || sc != nil || fn != nil { - nextValue = &descHolder - break - } - } - - di.value = nextValue -} - -// TenantIterator is a simple iterator to iterate over TenantInfoWithUsages. -type TenantIterator struct { - backing bytesIter - value *mtinfopb.TenantInfoWithUsage - err error -} - -// NewTenantIter creates a new TenantIterator for the backup metadata. -func (b *BackupMetadata) NewTenantIter( - ctx context.Context, -) bulk.Iterator[mtinfopb.TenantInfoWithUsage] { - backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstTenantsPrefix), b.enc, - false, b.kmsEnv) - it := TenantIterator{ - backing: backing, - } - it.Next() - return &it -} - -// Close closes the iterator. -func (ti *TenantIterator) Close() { - ti.backing.close() -} - -// Valid implements the Iterator interface. -func (ti *TenantIterator) Valid() (bool, error) { - if ti.err != nil { - return false, ti.err - } - return ti.value != nil, nil -} - -// Value implements the Iterator interface. -func (ti *TenantIterator) Value() mtinfopb.TenantInfoWithUsage { - if ti.value == nil { - return mtinfopb.TenantInfoWithUsage{} - } - return *ti.value -} - -// Next implements the Iterator interface. -func (ti *TenantIterator) Next() { - if ti.err != nil { - return - } - - wrapper := resultWrapper{} - ok := ti.backing.next(&wrapper) - if !ok { - if ti.backing.err() != nil { - ti.err = ti.backing.err() - } - ti.value = nil - return - } - - tenant := mtinfopb.TenantInfoWithUsage{} - - err := protoutil.Unmarshal(wrapper.value, &tenant) - if err != nil { - ti.err = err - return - } - - ti.value = &tenant -} - -// DescriptorRevisionIterator is a simple iterator to iterate over backuppb.BackupManifest_DescriptorRevisions. -type DescriptorRevisionIterator struct { - backing bytesIter - err error - value *backuppb.BackupManifest_DescriptorRevision -} - -// NewDescriptorChangesIter creates a new DescriptorChangesIterator for the backup metadata. -func (b *BackupMetadata) NewDescriptorChangesIter( - ctx context.Context, -) bulk.Iterator[*backuppb.BackupManifest_DescriptorRevision] { - if b.MVCCFilter == backuppb.MVCCFilter_Latest { - var backing []backuppb.BackupManifest_DescriptorRevision - return newSlicePointerIterator(backing) - } - - backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstDescsPrefix), b.enc, - false, b.kmsEnv) - dri := DescriptorRevisionIterator{ - backing: backing, - } - - dri.Next() - return &dri -} - -// Valid implements the Iterator interface. -func (dri *DescriptorRevisionIterator) Valid() (bool, error) { - if dri.err != nil { - return false, dri.err - } - return dri.value != nil, nil -} - -// Value implements the Iterator interface. -func (dri *DescriptorRevisionIterator) Value() *backuppb.BackupManifest_DescriptorRevision { - return dri.value -} - -// Close closes the iterator. -func (dri *DescriptorRevisionIterator) Close() { - dri.backing.close() -} - -// Next retrieves the next descriptor revision in the iterator. -// -// Next returns true if next element was successfully unmarshalled into -// revision, and false if there are no more elements or if an error was -// encountered. When Next returns false, the user should call the Err method to -// verify the existence of an error. -func (dri *DescriptorRevisionIterator) Next() { - if dri.err != nil { - return - } - - wrapper := resultWrapper{} - ok := dri.backing.next(&wrapper) - if !ok { - if err := dri.backing.err(); err != nil { - dri.err = err - } - - dri.value = nil - return - } - - nextRev, err := unmarshalWrapper(&wrapper) - if err != nil { - dri.err = err - return - } - - dri.value = &nextRev -} - -func unmarshalWrapper(wrapper *resultWrapper) (backuppb.BackupManifest_DescriptorRevision, error) { - var desc *descpb.Descriptor - if len(wrapper.value) > 0 { - desc = &descpb.Descriptor{} - err := protoutil.Unmarshal(wrapper.value, desc) - if err != nil { - return backuppb.BackupManifest_DescriptorRevision{}, err - } - } - - id, err := decodeDescSSTKey(wrapper.key.Key) - if err != nil { - return backuppb.BackupManifest_DescriptorRevision{}, err - } - - rev := backuppb.BackupManifest_DescriptorRevision{ - Desc: desc, - ID: id, - Time: wrapper.key.Timestamp, - } - return rev, nil -} - -// StatsIterator is a simple iterator to iterate over stats.TableStatisticProtos. -type StatsIterator struct { - backing bytesIter - value *stats.TableStatisticProto - err error -} - -// NewStatsIter creates a new StatsIterator for the backup metadata. -func (b *BackupMetadata) NewStatsIter( - ctx context.Context, -) bulk.Iterator[*stats.TableStatisticProto] { - backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstStatsPrefix), b.enc, - false, b.kmsEnv) - it := StatsIterator{ - backing: backing, - } - it.Next() - return &it -} - -// Close closes the iterator. -func (si *StatsIterator) Close() { - si.backing.close() -} - -// Valid implements the Iterator interface. -func (si *StatsIterator) Valid() (bool, error) { - if si.err != nil { - return false, si.err - } - return si.value != nil, nil -} - -// Value implements the Iterator interface. -func (si *StatsIterator) Value() *stats.TableStatisticProto { - return si.value -} - -func (si *StatsIterator) Next() { - if si.err != nil { - return - } - - wrapper := resultWrapper{} - ok := si.backing.next(&wrapper) - - if !ok { - if err := si.backing.err(); err != nil { - si.err = err - } - si.value = nil - return - } - - var s stats.TableStatisticProto - err := protoutil.Unmarshal(wrapper.value, &s) - if err != nil { - si.err = err - return - } - - si.value = &s -} - -type bytesIter struct { - Iter storage.SimpleMVCCIterator - - prefix []byte - useMVCCNext bool - iterError error -} - -func makeBytesIter( - ctx context.Context, - store cloud.ExternalStorage, - path string, - prefix []byte, - enc *jobspb.BackupEncryptionOptions, - useMVCCNext bool, - kmsEnv cloud.KMSEnv, -) bytesIter { - var encOpts *kvpb.FileEncryptionOptions - if enc != nil { - key, err := backupencryption.GetEncryptionKey(ctx, enc, kmsEnv) - if err != nil { - return bytesIter{iterError: err} - } - encOpts = &kvpb.FileEncryptionOptions{Key: key} - } - - iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: store, - FilePath: path}}, encOpts, iterOpts) - if err != nil { - return bytesIter{iterError: err} - } - - iter.SeekGE(storage.MakeMVCCMetadataKey(prefix)) - return bytesIter{ - Iter: iter, - prefix: prefix, - useMVCCNext: useMVCCNext, - } -} - -func (bi *bytesIter) next(resWrapper *resultWrapper) bool { - if bi.iterError != nil { - return false - } - - valid, err := bi.Iter.Valid() - if err != nil || !valid || !bytes.HasPrefix(bi.Iter.UnsafeKey().Key, bi.prefix) { - bi.iterError = err - return false - } - - key := bi.Iter.UnsafeKey() - resWrapper.key.Key = key.Key.Clone() - resWrapper.key.Timestamp = key.Timestamp - resWrapper.value = resWrapper.value[:0] - v, err := bi.Iter.UnsafeValue() - if err != nil { - bi.close() - bi.iterError = err - return false - } - resWrapper.value = append(resWrapper.value, v...) - - if bi.useMVCCNext { - bi.Iter.NextKey() - } else { - bi.Iter.Next() - } - return true -} - -func (bi *bytesIter) err() error { - return bi.iterError -} - -func (bi *bytesIter) close() { - if bi.Iter != nil { - bi.Iter.Close() - bi.Iter = nil - } -} - -type resultWrapper struct { - key storage.MVCCKey - value []byte -} - -type sliceIterator[T any] struct { - backingSlice []T - idx int -} - -var _ bulk.Iterator[*backuppb.BackupManifest_DescriptorRevision] = &sliceIterator[backuppb.BackupManifest_DescriptorRevision]{} - -func newSlicePointerIterator[T any](backing []T) *sliceIterator[T] { - return &sliceIterator[T]{ - backingSlice: backing, - } -} - -func (s *sliceIterator[T]) Valid() (bool, error) { - return s.idx < len(s.backingSlice), nil -} - -func (s *sliceIterator[T]) Value() *T { - if s.idx < len(s.backingSlice) { - return &s.backingSlice[s.idx] - } - - return nil -} - -func (s *sliceIterator[T]) Next() { - s.idx++ -} - -func (s *sliceIterator[T]) Close() { -} diff --git a/pkg/backup/backupinfo/backup_metadata_test.go b/pkg/backup/backupinfo/backup_metadata_test.go deleted file mode 100644 index 2575e54f142d..000000000000 --- a/pkg/backup/backupinfo/backup_metadata_test.go +++ /dev/null @@ -1,343 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -package backupinfo_test - -import ( - "context" - "fmt" - "sort" - "testing" - - "github.com/cockroachdb/cockroach/pkg/backup/backupbase" - "github.com/cockroachdb/cockroach/pkg/backup/backupencryption" - "github.com/cockroachdb/cockroach/pkg/backup/backupinfo" - "github.com/cockroachdb/cockroach/pkg/backup/backuppb" - "github.com/cockroachdb/cockroach/pkg/backup/backuptestutils" - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/blobs" - "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/isql" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/util/bulk" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/ioctx" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/mon" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/stretchr/testify/require" -) - -func TestMetadataSST(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - userfile := "userfile:///0" - tc, sqlDB, _, cleanupFn := backuptestutils.StartBackupRestoreTestCluster(t, - backuptestutils.SingleNode, - backuptestutils.WithBank(1), - backuptestutils.WithParams(base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - DefaultTestTenant: base.TestControlsTenantsExplicitly, - }, - })) - defer cleanupFn() - - sqlDB.Exec(t, `SET CLUSTER SETTING kv.bulkio.write_metadata_sst.enabled = true`) - - // Check that backup metadata is correct on full cluster backup. - sqlDB.Exec(t, `BACKUP INTO $1`, userfile) - backupPath := userfile + getBackupPath(t, sqlDB, userfile) - checkMetadata(ctx, t, tc, backupPath) - - // Check for correct backup metadata on incremental backup with revision - // history. - sqlDB.Exec(t, `CREATE TABLE data.foo(k INT, v INT)`) - sqlDB.Exec(t, `CREATE INDEX idx ON data.bank (balance)`) - sqlDB.Exec(t, `CREATE DATABASE emptydb`) - sqlDB.Exec(t, `CREATE TABLE emptydb.bar(k INT, v INT)`) - sqlDB.Exec(t, `DROP DATABASE emptydb`) - - sqlDB.Exec(t, `BACKUP INTO $1 WITH revision_history`, userfile) - checkMetadata(ctx, t, tc, backupPath) - - // Check for correct backup metadata on single table backups. - userfile1 := "userfile:///1" - sqlDB.Exec(t, `BACKUP TABLE data.bank INTO $1 WITH revision_history`, userfile1) - backupPath1 := userfile1 + getBackupPath(t, sqlDB, userfile1) - checkMetadata(ctx, t, tc, backupPath1) - - // Check for correct backup metadata on tenant backups. - userfile2 := "userfile:///2" - _, err := tc.Servers[0].TenantController().StartTenant(ctx, base.TestTenantArgs{TenantID: roachpb.MustMakeTenantID(10)}) - require.NoError(t, err) - sqlDB.Exec(t, `BACKUP TENANT 10 INTO $1`, userfile2) - backupPath2 := userfile2 + getBackupPath(t, sqlDB, userfile2) - checkMetadata(ctx, t, tc, backupPath2) -} - -func checkMetadata( - ctx context.Context, t *testing.T, tc *testcluster.TestCluster, backupLoc string, -) { - t.Helper() - store, err := cloud.ExternalStorageFromURI( - ctx, - backupLoc, - base.ExternalIODirConfig{}, - tc.Servers[0].ClusterSettings(), - blobs.TestEmptyBlobClientFactory, - username.RootUserName(), - tc.Servers[0].InternalDB().(isql.DB), - nil, /* limiters */ - cloud.NilMetrics, - ) - if err != nil { - t.Fatal(err) - } - m, err := testingReadBackupManifest(ctx, store, backupbase.BackupManifestName) - if err != nil { - t.Fatal(err) - } - - srv := tc.Servers[0] - execCfg := srv.ExecutorConfig().(sql.ExecutorConfig) - kmsEnv := backupencryption.MakeBackupKMSEnv(srv.ClusterSettings(), &base.ExternalIODirConfig{}, - execCfg.InternalDB, username.RootUserName()) - bm, err := backupinfo.NewBackupMetadata(ctx, store, backupinfo.MetadataSSTName, - nil /* encryption */, &kmsEnv) - if err != nil { - t.Fatal(err) - } - - checkManifest(t, m, bm) - checkDescriptorChanges(ctx, t, m, bm) - checkDescriptors(ctx, t, m, bm) - checkSpans(ctx, t, m, bm) - // Don't check introduced spans on the first backup. - if m.StartTime != (hlc.Timestamp{}) { - checkIntroducedSpans(ctx, t, m, bm) - } - checkFiles(ctx, t, m, bm) - checkTenants(ctx, t, m, bm) - checkStats(ctx, t, store, m, bm, &kmsEnv) -} - -func checkManifest(t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata) { - expectedManifest := *m - expectedManifest.Descriptors = nil - expectedManifest.DescriptorChanges = nil - expectedManifest.Files = nil - expectedManifest.Spans = nil - expectedManifest.IntroducedSpans = nil - expectedManifest.StatisticsFilenames = nil - expectedManifest.Tenants = nil - - require.Equal(t, expectedManifest, bm.BackupManifest) -} - -func checkDescriptors( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, -) { - var metaDescs []descpb.Descriptor - - it := bm.NewDescIter(ctx) - defer it.Close() - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - - metaDescs = append(metaDescs, *it.Value()) - } - - require.Equal(t, m.Descriptors, metaDescs) -} - -func checkDescriptorChanges( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, -) { - var metaRevs []backuppb.BackupManifest_DescriptorRevision - it := bm.NewDescriptorChangesIter(ctx) - defer it.Close() - - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - metaRevs = append(metaRevs, *it.Value()) - } - - // Descriptor Changes are sorted by time in the manifest. - sort.Slice(metaRevs, func(i, j int) bool { - return metaRevs[i].Time.Less(metaRevs[j].Time) - }) - - require.Equal(t, m.DescriptorChanges, metaRevs) -} - -func checkFiles( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, -) { - var metaFiles []backuppb.BackupManifest_File - it, err := bm.NewFileIter(ctx) - if err != nil { - t.Fatal(err) - } - defer it.Close() - - for ; ; it.Next() { - ok, err := it.Valid() - if err != nil { - t.Fatal(err) - } - if !ok { - break - } - - metaFiles = append(metaFiles, *it.Value()) - } - - require.Equal(t, m.Files, metaFiles) -} - -func checkSpans( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, -) { - var metaSpans []roachpb.Span - it := bm.NewSpanIter(ctx) - defer it.Close() - - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - - metaSpans = append(metaSpans, it.Value()) - } - - require.Equal(t, m.Spans, metaSpans) -} - -func checkIntroducedSpans( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, -) { - var metaSpans []roachpb.Span - it := bm.NewIntroducedSpanIter(ctx) - defer it.Close() - - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - metaSpans = append(metaSpans, it.Value()) - } - - require.Equal(t, m.IntroducedSpans, metaSpans) -} - -func checkTenants( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, -) { - var metaTenants []mtinfopb.TenantInfoWithUsage - it := bm.NewTenantIter(ctx) - defer it.Close() - - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - - metaTenants = append(metaTenants, it.Value()) - } - - require.Equal(t, m.Tenants, metaTenants) -} - -func checkStats( - ctx context.Context, - t *testing.T, - store cloud.ExternalStorage, - m *backuppb.BackupManifest, - bm *backupinfo.BackupMetadata, - kmsEnv cloud.KMSEnv, -) { - expectedStats, err := backupinfo.GetStatisticsFromBackup(ctx, store, nil, kmsEnv, *m) - if err != nil { - t.Fatal(err) - } - if len(expectedStats) == 0 { - expectedStats = nil - } - - sort.Slice(expectedStats, func(i, j int) bool { - return expectedStats[i].TableID < expectedStats[j].TableID || - (expectedStats[i].TableID == expectedStats[j].TableID && expectedStats[i].StatisticID < expectedStats[j].StatisticID) - }) - - it := bm.NewStatsIter(ctx) - defer it.Close() - metaStats, err := bulk.CollectToSlice(it) - if err != nil { - t.Fatal(err) - } - - require.Equal(t, expectedStats, metaStats) -} - -// Gets the first backup path in a userfile path. -// Note: the tests in this file expects only one backup in the path so only fetches the first backup -func getBackupPath(t *testing.T, db *sqlutils.SQLRunner, userfile string) string { - rows := db.Query(t, "SHOW BACKUPS IN $1", userfile) - var result struct{ path string } - require.True(t, rows.Next(), fmt.Sprintf("Could not find backup path in %s", userfile)) - require.NoError(t, rows.Scan(&result.path)) - return result.path -} - -func testingReadBackupManifest( - ctx context.Context, store cloud.ExternalStorage, file string, -) (*backuppb.BackupManifest, error) { - r, _, err := store.ReadFile(ctx, file, cloud.ReadOptions{NoFileSize: true}) - if err != nil { - return nil, err - } - defer r.Close(ctx) - - bytes, err := ioctx.ReadAll(ctx, r) - if err != nil { - return nil, err - } - if backupinfo.IsGZipped(bytes) { - descBytes, err := backupinfo.DecompressData(ctx, mon.NewStandaloneUnlimitedAccount(), bytes) - if err != nil { - return nil, err - } - bytes = descBytes - } - - var m backuppb.BackupManifest - if err := protoutil.Unmarshal(bytes, &m); err != nil { - return nil, err - } - return &m, nil -} diff --git a/pkg/backup/backupinfo/desc_sst.go b/pkg/backup/backupinfo/desc_sst.go new file mode 100644 index 000000000000..81bd8c419622 --- /dev/null +++ b/pkg/backup/backupinfo/desc_sst.go @@ -0,0 +1,342 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package backupinfo + +import ( + "bytes" + "context" + "fmt" + "sort" + + "github.com/cockroachdb/cockroach/pkg/backup/backuppb" + "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/bulk" + "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +const ( + // BackupMetadataDescriptorsListPath is the name of the SST file containing + // the BackupManifest_Descriptors or BackupManifest_DescriptorRevisions of the + // backup. This file is always written in conjunction with the + // `BACKUP_METADATA`. + BackupMetadataDescriptorsListPath = "descriptorslist.sst" + sstDescsPrefix = "desc/" +) + +// WriteDescsSST is responsible for writing the SST containing the Descriptor +// and DescriptorChanges field of the input BackupManifest. If DescriptorChanges +// is non-empty, then the descriptor changes will be written to the SST with the +// MVCC timestamp equal to the revision time. Otherwise, contents of the +// Descriptors field will be written to the SST with an empty MVCC timestamp. +func WriteDescsSST( + ctx context.Context, + m *backuppb.BackupManifest, + dest cloud.ExternalStorage, + enc *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, + path string, +) error { + w, err := makeWriter(ctx, dest, path, enc, kmsEnv) + if err != nil { + return err + } + defer w.Close() + descSST := storage.MakeTransportSSTWriter(ctx, dest.Settings(), w) + defer descSST.Close() + + if err := writeDescsToMetadata(ctx, descSST, m); err != nil { + return err + } + + if err := descSST.Finish(); err != nil { + return err + } + + return w.Close() +} + +func DescChangesLess( + left *backuppb.BackupManifest_DescriptorRevision, + right *backuppb.BackupManifest_DescriptorRevision, +) bool { + if left.ID != right.ID { + return left.ID < right.ID + } + + return !left.Time.Less(right.Time) +} + +func writeDescsToMetadata( + ctx context.Context, sst storage.SSTWriter, m *backuppb.BackupManifest, +) error { + // Add descriptors from revisions if available, Descriptors if not. + if len(m.DescriptorChanges) > 0 { + sort.Slice(m.DescriptorChanges, func(i, j int) bool { + return DescChangesLess(&m.DescriptorChanges[i], &m.DescriptorChanges[j]) + }) + for _, i := range m.DescriptorChanges { + k := encodeDescSSTKey(i.ID) + var b []byte + if i.Desc != nil { + t, _, _, _, _ := descpb.GetDescriptors(i.Desc) + if t == nil || !t.Dropped() { + bytes, err := protoutil.Marshal(i.Desc) + if err != nil { + return err + } + b = bytes + } + } + if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: i.Time}, b); err != nil { + return err + } + + } + } else { + sort.Slice(m.Descriptors, func(i, j int) bool { + return descID(m.Descriptors[i]) < descID(m.Descriptors[j]) + }) + for _, i := range m.Descriptors { + id := descID(i) + k := encodeDescSSTKey(id) + b, err := protoutil.Marshal(&i) + if err != nil { + return err + } + + // Put descriptors at start time. For non-rev backups this timestamp + // doesn't matter. For the special case where there were no descriptor + // changes in an incremental backup, it's helpful to have existing + // descriptors at the start time, so we don't have to look back further + // than the very last backup. + if m.StartTime.IsEmpty() || m.MVCCFilter == backuppb.MVCCFilter_Latest { + if err := sst.PutUnversioned(k, b); err != nil { + return err + } + } else { + if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: m.StartTime}, b); err != nil { + return err + } + } + } + } + return nil +} + +func descID(in descpb.Descriptor) descpb.ID { + switch i := in.Union.(type) { + case *descpb.Descriptor_Table: + return i.Table.ID + case *descpb.Descriptor_Database: + return i.Database.ID + case *descpb.Descriptor_Type: + return i.Type.ID + case *descpb.Descriptor_Schema: + return i.Schema.ID + case *descpb.Descriptor_Function: + return i.Function.ID + default: + panic(fmt.Sprintf("unknown desc %T", in)) + } +} + +func encodeDescSSTKey(id descpb.ID) roachpb.Key { + return roachpb.Key(encoding.EncodeUvarintAscending([]byte(sstDescsPrefix), uint64(id))) +} + +func deprefix(key roachpb.Key, prefix string) (roachpb.Key, error) { + if !bytes.HasPrefix(key, []byte(prefix)) { + return nil, errors.Errorf("malformed key missing expected prefix %s: %q", prefix, key) + } + return key[len(prefix):], nil +} + +func decodeDescSSTKey(key roachpb.Key) (descpb.ID, error) { + key, err := deprefix(key, sstDescsPrefix) + if err != nil { + return 0, err + } + _, id, err := encoding.DecodeUvarintAscending(key) + return descpb.ID(id), err +} + +var _ bulk.Iterator[*backuppb.BackupManifest_DescriptorRevision] = &sliceIterator[backuppb.BackupManifest_DescriptorRevision]{} + +// DescriptorRevisionIterator is a simple iterator to iterate over backuppb.BackupManifest_DescriptorRevisions. +type DescriptorRevisionIterator struct { + backing bytesIter + err error + value *backuppb.BackupManifest_DescriptorRevision +} + +func (dri *DescriptorRevisionIterator) Valid() (bool, error) { + if dri.err != nil { + return false, dri.err + } + return dri.value != nil, nil +} + +// Value implements the Iterator interface. +func (dri *DescriptorRevisionIterator) Value() *backuppb.BackupManifest_DescriptorRevision { + return dri.value +} + +// Close closes the iterator. +func (dri *DescriptorRevisionIterator) Close() { + dri.backing.close() +} + +// Next retrieves the next descriptor revision in the iterator. +// +// Next returns true if next element was successfully unmarshalled into +// revision, and false if there are no more elements or if an error was +// encountered. When Next returns false, the user should call the Err method to +// verify the existence of an error. +func (dri *DescriptorRevisionIterator) Next() { + if dri.err != nil { + return + } + + wrapper := resultWrapper{} + ok := dri.backing.next(&wrapper) + if !ok { + if err := dri.backing.err(); err != nil { + dri.err = err + } + + dri.value = nil + return + } + + nextRev, err := unmarshalWrapper(&wrapper) + if err != nil { + dri.err = err + return + } + + dri.value = &nextRev +} + +func unmarshalWrapper(wrapper *resultWrapper) (backuppb.BackupManifest_DescriptorRevision, error) { + var desc *descpb.Descriptor + if len(wrapper.value) > 0 { + desc = &descpb.Descriptor{} + err := protoutil.Unmarshal(wrapper.value, desc) + if err != nil { + return backuppb.BackupManifest_DescriptorRevision{}, err + } + } + + id, err := decodeDescSSTKey(wrapper.key.Key) + if err != nil { + return backuppb.BackupManifest_DescriptorRevision{}, err + } + + rev := backuppb.BackupManifest_DescriptorRevision{ + Desc: desc, + ID: id, + Time: wrapper.key.Timestamp, + } + return rev, nil +} + +// NewDescIter creates a new Iterator over Descriptors. +func (f *IterFactory) NewDescIter(ctx context.Context) bulk.Iterator[*descpb.Descriptor] { + if f.m.HasExternalManifestSSTs { + backing := makeBytesIter(ctx, f.store, f.descriptorSSTPath, []byte(sstDescsPrefix), f.encryption, true, f.kmsEnv) + it := DescIterator{ + backing: backing, + } + it.Next() + return &it + } + + return newSlicePointerIterator(f.m.Descriptors) +} + +// Close closes the iterator. +func (di *DescIterator) Close() { + di.backing.close() +} + +// Valid implements the Iterator interface. +func (di *DescIterator) Valid() (bool, error) { + if di.err != nil { + return false, di.err + } + return di.value != nil, nil +} + +// Value implements the Iterator interface. +func (di *DescIterator) Value() *descpb.Descriptor { + return di.value +} + +// Next implements the Iterator interface. +func (di *DescIterator) Next() { + if di.err != nil { + return + } + + wrapper := resultWrapper{} + var nextValue *descpb.Descriptor + descHolder := descpb.Descriptor{} + for di.backing.next(&wrapper) { + err := protoutil.Unmarshal(wrapper.value, &descHolder) + if err != nil { + di.err = err + return + } + + tbl, db, typ, sc, fn := descpb.GetDescriptors(&descHolder) + if tbl != nil || db != nil || typ != nil || sc != nil || fn != nil { + nextValue = &descHolder + break + } + } + + di.value = nextValue +} + +// DescIterator is a simple iterator to iterate over descpb.Descriptors. +type DescIterator struct { + backing bytesIter + value *descpb.Descriptor + err error +} + +// NewDescriptorChangesIter creates a new Iterator over +// BackupManifest_DescriptorRevisions. It is assumed that descriptor changes are +// sorted by DescChangesLess. +func (f *IterFactory) NewDescriptorChangesIter( + ctx context.Context, +) bulk.Iterator[*backuppb.BackupManifest_DescriptorRevision] { + if f.m.HasExternalManifestSSTs { + if f.m.MVCCFilter == backuppb.MVCCFilter_Latest { + // If the manifest is backuppb.MVCCFilter_Latest, then return an empty + // iterator for descriptor changes. + var backing []backuppb.BackupManifest_DescriptorRevision + return newSlicePointerIterator(backing) + } + + backing := makeBytesIter(ctx, f.store, f.descriptorSSTPath, []byte(sstDescsPrefix), f.encryption, + false, f.kmsEnv) + dri := DescriptorRevisionIterator{ + backing: backing, + } + + dri.Next() + return &dri + } + + return newSlicePointerIterator(f.m.DescriptorChanges) +} diff --git a/pkg/backup/backupinfo/external_sst_util.go b/pkg/backup/backupinfo/external_sst_util.go new file mode 100644 index 000000000000..78ec61d56d5f --- /dev/null +++ b/pkg/backup/backupinfo/external_sst_util.go @@ -0,0 +1,169 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package backupinfo + +import ( + "bytes" + "context" + "io" + + "github.com/cockroachdb/cockroach/pkg/backup/backupencryption" + "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" + "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/storage" +) + +func makeWriter( + ctx context.Context, + dest cloud.ExternalStorage, + filename string, + enc *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, +) (io.WriteCloser, error) { + w, err := dest.Writer(ctx, filename) + if err != nil { + return nil, err + } + + if enc != nil { + key, err := backupencryption.GetEncryptionKey(ctx, enc, kmsEnv) + if err != nil { + return nil, err + } + encW, err := storageccl.EncryptingWriter(w, key) + if err != nil { + return nil, err + } + w = encW + } + return w, nil +} + +var iterOpts = storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsOnly, + LowerBound: keys.LocalMax, + UpperBound: keys.MaxKey, +} + +type sliceIterator[T any] struct { + backingSlice []T + idx int +} + +func newSlicePointerIterator[T any](backing []T) *sliceIterator[T] { + return &sliceIterator[T]{ + backingSlice: backing, + } +} + +func (s *sliceIterator[T]) Valid() (bool, error) { + return s.idx < len(s.backingSlice), nil +} + +func (s *sliceIterator[T]) Value() *T { + if s.idx < len(s.backingSlice) { + return &s.backingSlice[s.idx] + } + + return nil +} + +func (s *sliceIterator[T]) Next() { + s.idx++ +} + +func (s *sliceIterator[T]) Close() { +} + +type bytesIter struct { + Iter storage.SimpleMVCCIterator + + prefix []byte + useMVCCNext bool + iterError error +} + +type resultWrapper struct { + key storage.MVCCKey + value []byte +} + +func makeBytesIter( + ctx context.Context, + store cloud.ExternalStorage, + path string, + prefix []byte, + enc *jobspb.BackupEncryptionOptions, + useMVCCNext bool, + kmsEnv cloud.KMSEnv, +) bytesIter { + var encOpts *kvpb.FileEncryptionOptions + if enc != nil { + key, err := backupencryption.GetEncryptionKey(ctx, enc, kmsEnv) + if err != nil { + return bytesIter{iterError: err} + } + encOpts = &kvpb.FileEncryptionOptions{Key: key} + } + + iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: store, + FilePath: path}}, encOpts, iterOpts) + if err != nil { + return bytesIter{iterError: err} + } + + iter.SeekGE(storage.MakeMVCCMetadataKey(prefix)) + return bytesIter{ + Iter: iter, + prefix: prefix, + useMVCCNext: useMVCCNext, + } +} + +func (bi *bytesIter) next(resWrapper *resultWrapper) bool { + if bi.iterError != nil { + return false + } + + valid, err := bi.Iter.Valid() + if err != nil || !valid || !bytes.HasPrefix(bi.Iter.UnsafeKey().Key, bi.prefix) { + bi.iterError = err + return false + } + + key := bi.Iter.UnsafeKey() + resWrapper.key.Key = key.Key.Clone() + resWrapper.key.Timestamp = key.Timestamp + resWrapper.value = resWrapper.value[:0] + v, err := bi.Iter.UnsafeValue() + if err != nil { + bi.close() + bi.iterError = err + return false + } + resWrapper.value = append(resWrapper.value, v...) + + if bi.useMVCCNext { + bi.Iter.NextKey() + } else { + bi.Iter.Next() + } + return true +} + +func (bi *bytesIter) err() error { + return bi.iterError +} + +func (bi *bytesIter) close() { + if bi.Iter != nil { + bi.Iter.Close() + bi.Iter = nil + } +} diff --git a/pkg/backup/backupinfo/file_sst.go b/pkg/backup/backupinfo/file_sst.go new file mode 100644 index 000000000000..699240825fd5 --- /dev/null +++ b/pkg/backup/backupinfo/file_sst.go @@ -0,0 +1,176 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package backupinfo + +import ( + "context" + "sort" + "strings" + + "github.com/cockroachdb/cockroach/pkg/backup/backuppb" + "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" + "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" +) + +const ( + // BackupMetadataFilesListPath is the name of the SST file containing the + // BackupManifest_Files of the backup. This file is always written in + // conjunction with the `BACKUP_METADATA`. + BackupMetadataFilesListPath = "filelist.sst" + sstFilesPrefix = "file/" +) + +// WriteFilesListSST is responsible for constructing and writing the +// filePathInfo to dest. This file contains the `BackupManifest_Files` of the +// backup. +func WriteFilesListSST( + ctx context.Context, + dest cloud.ExternalStorage, + enc *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, + manifest *backuppb.BackupManifest, + filePathInfo string, +) error { + return writeFilesSST(ctx, manifest, dest, enc, kmsEnv, filePathInfo) +} + +func writeFilesSST( + ctx context.Context, + m *backuppb.BackupManifest, + dest cloud.ExternalStorage, + enc *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, + fileInfoPath string, +) error { + w, err := makeWriter(ctx, dest, fileInfoPath, enc, kmsEnv) + if err != nil { + return err + } + defer w.Close() + fileSST := storage.MakeTransportSSTWriter(ctx, dest.Settings(), w) + defer fileSST.Close() + + // Sort and write all of the files into a single file info SST. + sort.Slice(m.Files, func(i, j int) bool { + return FileCmp(m.Files[i], m.Files[j]) < 0 + }) + + for i := range m.Files { + file := m.Files[i] + b, err := protoutil.Marshal(&file) + if err != nil { + return err + } + if err := fileSST.PutUnversioned(encodeFileSSTKey(file.Span.Key, file.Path), b); err != nil { + return err + } + } + + err = fileSST.Finish() + if err != nil { + return err + } + return w.Close() +} + +func encodeFileSSTKey(spanStart roachpb.Key, filename string) roachpb.Key { + buf := make([]byte, 0) + buf = encoding.EncodeBytesAscending(buf, spanStart) + return roachpb.Key(encoding.EncodeStringAscending(buf, filename)) +} + +// FileCmp gives an ordering to two backuppb.BackupManifest_File. +func FileCmp(left backuppb.BackupManifest_File, right backuppb.BackupManifest_File) int { + if cmp := left.Span.Key.Compare(right.Span.Key); cmp != 0 { + return cmp + } + + return strings.Compare(left.Path, right.Path) +} + +// NewFileSSTIter creates a new FileIterator to iterate over the storeFile. +// It is the caller's responsibility to Close() the returned iterator. +func NewFileSSTIter( + ctx context.Context, storeFile storageccl.StoreFile, encOpts *kvpb.FileEncryptionOptions, +) (*FileIterator, error) { + return newFileSSTIter(ctx, []storageccl.StoreFile{storeFile}, encOpts) +} + +// FileIterator is a simple iterator to iterate over backuppb.BackupManifest_File. +type FileIterator struct { + mergedIterator storage.SimpleMVCCIterator + err error + file *backuppb.BackupManifest_File +} + +func newFileSSTIter( + ctx context.Context, storeFiles []storageccl.StoreFile, encOpts *kvpb.FileEncryptionOptions, +) (*FileIterator, error) { + iter, err := storageccl.ExternalSSTReader(ctx, storeFiles, encOpts, iterOpts) + if err != nil { + return nil, err + } + iter.SeekGE(storage.MVCCKey{}) + fi := &FileIterator{mergedIterator: iter} + fi.Next() + return fi, nil +} + +// Close closes the iterator. +func (fi *FileIterator) Close() { + fi.mergedIterator.Close() +} + +// Valid indicates whether or not the iterator is pointing to a valid value. +func (fi *FileIterator) Valid() (bool, error) { + if fi.err != nil { + return false, fi.err + } + + return fi.file != nil, nil +} + +// Value implements the Iterator interface. +func (fi *FileIterator) Value() *backuppb.BackupManifest_File { + return fi.file +} + +// Next implements the Iterator interface. +func (fi *FileIterator) Next() { + if fi.err != nil { + return + } + + if ok, err := fi.mergedIterator.Valid(); !ok { + if err != nil { + fi.err = err + } + fi.file = nil + return + } + + v, err := fi.mergedIterator.UnsafeValue() + if err != nil { + fi.err = err + return + } + + file := &backuppb.BackupManifest_File{} + err = protoutil.Unmarshal(v, file) + if err != nil { + fi.err = err + return + } + + fi.file = file + fi.mergedIterator.Next() +} diff --git a/pkg/backup/backupinfo/manifest_handling.go b/pkg/backup/backupinfo/manifest_handling.go index 35f138be1f51..6dba1d8c5ed5 100644 --- a/pkg/backup/backupinfo/manifest_handling.go +++ b/pkg/backup/backupinfo/manifest_handling.go @@ -86,15 +86,6 @@ const ( BackupProgressDirectory = "progress" ) -// WriteMetadataSST controls if we write the experimental new format BACKUP -// metadata file. -var WriteMetadataSST = settings.RegisterBoolSetting( - settings.ApplicationLevel, - "kv.bulkio.write_metadata_sst.enabled", - "write experimental new format BACKUP metadata file", - metamorphic.ConstantWithTestBool("write-metadata-sst", false), -) - // WriteMetadataWithExternalSSTsEnabled controls if we write a `BACKUP_METADATA` // file along with external SSTs containing lists of `BackupManifest_Files` and // descriptors. This new format of metadata is written in addition to the @@ -1657,47 +1648,6 @@ func (f *IterFactory) NewFileIter( return newSlicePointerIterator(f.m.Files), nil } -// NewDescIter creates a new Iterator over Descriptors. -func (f *IterFactory) NewDescIter(ctx context.Context) bulk.Iterator[*descpb.Descriptor] { - if f.m.HasExternalManifestSSTs { - backing := makeBytesIter(ctx, f.store, f.descriptorSSTPath, []byte(sstDescsPrefix), f.encryption, true, f.kmsEnv) - it := DescIterator{ - backing: backing, - } - it.Next() - return &it - } - - return newSlicePointerIterator(f.m.Descriptors) -} - -// NewDescriptorChangesIter creates a new Iterator over -// BackupManifest_DescriptorRevisions. It is assumed that descriptor changes are -// sorted by DescChangesLess. -func (f *IterFactory) NewDescriptorChangesIter( - ctx context.Context, -) bulk.Iterator[*backuppb.BackupManifest_DescriptorRevision] { - if f.m.HasExternalManifestSSTs { - if f.m.MVCCFilter == backuppb.MVCCFilter_Latest { - // If the manifest is backuppb.MVCCFilter_Latest, then return an empty - // iterator for descriptor changes. - var backing []backuppb.BackupManifest_DescriptorRevision - return newSlicePointerIterator(backing) - } - - backing := makeBytesIter(ctx, f.store, f.descriptorSSTPath, []byte(sstDescsPrefix), f.encryption, - false, f.kmsEnv) - dri := DescriptorRevisionIterator{ - backing: backing, - } - - dri.Next() - return &dri - } - - return newSlicePointerIterator(f.m.DescriptorChanges) -} - // GetBackupManifestIterFactories constructs a mapping from the idx of the // backup layer to an IterFactory. func GetBackupManifestIterFactories( diff --git a/pkg/backup/full_cluster_backup_restore_test.go b/pkg/backup/full_cluster_backup_restore_test.go index 9ad2c2378220..bb7994cd2a9c 100644 --- a/pkg/backup/full_cluster_backup_restore_test.go +++ b/pkg/backup/full_cluster_backup_restore_test.go @@ -638,12 +638,6 @@ func TestClusterRestoreFailCleanup(t *testing.T) { isBackupOfSystemTenant := tcBackup.ApplicationLayer(0).Codec().ForSystemTenant() - // This test flakes due to - // https://github.com/cockroachdb/cockroach/issues/86806. Instead of skipping - // the test all together, setting the cluster setting to false which triggers - // the failure. - sqlDB.Exec(t, "SET CLUSTER SETTING kv.bulkio.write_metadata_sst.enabled=false") - // Setup the system systemTablesToVerify to ensure that they are copied to the new cluster. // Populate system.users. numBatches := 100 diff --git a/pkg/backup/show.go b/pkg/backup/show.go index af8e4ad2a765..4d30a13cb8a2 100644 --- a/pkg/backup/show.go +++ b/pkg/backup/show.go @@ -43,7 +43,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -127,54 +126,8 @@ func (m manifestInfoReader) showBackup( return nil } -type metadataSSTInfoReader struct{} - var _ backupInfoReader = manifestInfoReader{} -func (m metadataSSTInfoReader) header() colinfo.ResultColumns { - return colinfo.ResultColumns{ - {Name: "file", Typ: types.String}, - {Name: "key", Typ: types.String}, - {Name: "detail", Typ: types.Jsonb}, - } -} - -func (m metadataSSTInfoReader) showBackup( - ctx context.Context, - _ *mon.BoundAccount, - mkStore cloud.ExternalStorageFromURIFactory, - info backupInfo, - user username.SQLUsername, - kmsEnv cloud.KMSEnv, - resultsCh chan<- tree.Datums, -) error { - filename := backupinfo.MetadataSSTName - push := func(_, readable string, value json.JSON) error { - val := tree.DNull - if value != nil { - val = tree.NewDJSON(value) - } - select { - case <-ctx.Done(): - return ctx.Err() - case resultsCh <- []tree.Datum{tree.NewDString(filename), tree.NewDString(readable), val}: - return nil - } - } - for _, uri := range info.defaultURIs { - store, err := mkStore(ctx, uri, user) - if err != nil { - return errors.Wrapf(err, "creating external store") - } - defer store.Close() - if err := backupinfo.DebugDumpMetadataSST(ctx, store, filename, info.enc, - kmsEnv, push); err != nil { - return err - } - } - return nil -} - func showBackupTypeCheck( ctx context.Context, stmt tree.Statement, p sql.PlanHookState, ) (matched bool, header colinfo.ResultColumns, _ error) { @@ -500,9 +453,7 @@ you must pass the 'encryption_info_dir' parameter that points to the directory o func getBackupInfoReader(p sql.PlanHookState, showStmt *tree.ShowBackup) backupInfoReader { var infoReader backupInfoReader - if showStmt.Options.DebugMetadataSST { - infoReader = metadataSSTInfoReader{} - } else if showStmt.Options.AsJson { + if showStmt.Options.AsJson { infoReader = manifestInfoReader{shower: jsonShower} } else { var shower backupShower @@ -559,18 +510,10 @@ func checkBackupFiles( // Check metadata files. Note: we do not check locality aware backup // metadata files ( prefixed with `backupPartitionDescriptorPrefix`) , as // they're validated in resolveBackupManifests. - for _, metaFile := range []string{ - backupinfo.FileInfoPath, - backupinfo.MetadataSSTName, - backupbase.BackupManifestName + backupinfo.BackupManifestChecksumSuffix} { - if _, err := defaultStore.Size(ctx, metaFile); err != nil { - if metaFile == backupinfo.FileInfoPath || metaFile == backupinfo.MetadataSSTName { - log.Warningf(ctx, `%v not found. This is only relevant if kv.bulkio.write_metadata_sst.enabled = true`, metaFile) - continue - } - return nil, errors.Wrapf(err, "Error checking metadata file %s/%s", - info.defaultURIs[layer], metaFile) - } + metaFile := backupbase.BackupManifestName + backupinfo.BackupManifestChecksumSuffix + if _, err := defaultStore.Size(ctx, metaFile); err != nil { + return nil, errors.Wrapf(err, "Error checking metadata file %s/%s", + info.defaultURIs[layer], metaFile) } // Check stat files. for _, statFile := range info.manifests[layer].StatisticsFilenames { diff --git a/pkg/backup/tenant_backup_nemesis_test.go b/pkg/backup/tenant_backup_nemesis_test.go index 687a99dca9d3..d82bd1836d00 100644 --- a/pkg/backup/tenant_backup_nemesis_test.go +++ b/pkg/backup/tenant_backup_nemesis_test.go @@ -211,9 +211,6 @@ func TestTenantBackupNemesis(t *testing.T) { // the ID here because the IDs are built into the test // certificates. hostSQLDB.Exec(t, "SELECT crdb_internal.create_tenant(12)") - // Disabled until #113852 is fixed or the feature is removed. - hostSQLDB.Exec(t, "SET CLUSTER SETTING kv.bulkio.write_metadata_sst.enabled=false") - _, err = tenant10Conn.Exec("SET CLUSTER SETTING kv.bulkio.write_metadata_sst.enabled=false") require.NoError(t, err) rng, _ := randutil.NewPseudoRand() diff --git a/pkg/backup/testdata/backup-restore/in-progress-import-rollback b/pkg/backup/testdata/backup-restore/in-progress-import-rollback index 2dcd6137de23..ff1a0c70a874 100644 --- a/pkg/backup/testdata/backup-restore/in-progress-import-rollback +++ b/pkg/backup/testdata/backup-restore/in-progress-import-rollback @@ -30,10 +30,6 @@ exec-sql SET CLUSTER SETTING jobs.debug.pausepoints = 'import.after_ingest'; ---- -exec-sql -SET CLUSTER SETTING kv.bulkio.write_metadata_sst.enabled = false; ----- - exec-sql EXPORT INTO CSV 'nodelocal://1/export1/' FROM SELECT * FROM baz; ---- @@ -231,10 +227,6 @@ exec-sql SET CLUSTER SETTING jobs.debug.pausepoints = 'import.after_ingest'; ---- -exec-sql -SET CLUSTER SETTING kv.bulkio.write_metadata_sst.enabled = false; ----- - # Pause the import job, in order to back up the importing data. import expect-pausepoint tag=b IMPORT INTO foo2 (i,s) CSV DATA ('nodelocal://1/export1/export*-n*.0.csv') diff --git a/pkg/backup/testdata/backup-restore/metadata b/pkg/backup/testdata/backup-restore/metadata index 574d88865423..a463029d869f 100644 --- a/pkg/backup/testdata/backup-restore/metadata +++ b/pkg/backup/testdata/backup-restore/metadata @@ -5,10 +5,6 @@ new-cluster name=s ---- -exec-sql -SET CLUSTER SETTING kv.bulkio.write_metadata_sst.enabled = true; ----- - # Automatic collection is disabled to avoid flakiness. exec-sql SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false; diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index 5e80a39b8594..b4b9fce7f15e 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -251,6 +251,7 @@ var retiredSettings = map[InternalKey]struct{}{ "storage.single_delete.crash_on_invariant_violation.enabled": {}, "storage.single_delete.crash_on_ineffectual.enabled": {}, "bulkio.backup.elide_common_prefix.enabled": {}, + "kv.bulkio.write_metadata_sst.enabled": {}, } // grandfatheredDefaultSettings is the list of "grandfathered" existing sql.defaults diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index bd84cf4e21a1..35f58128c311 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -986,7 +986,7 @@ func (u *sqlSymUnion) doBlockOption() tree.DoBlockOption { %token CURRENT_ROLE CURRENT_TIME CURRENT_TIMESTAMP %token CURRENT_USER CURSOR CYCLE -%token DATA DATABASE DATABASES DATE DAY DEBUG_IDS DEC DEBUG_DUMP_METADATA_SST DECIMAL DEFAULT DEFAULTS DEFINER +%token DATA DATABASE DATABASES DATE DAY DEBUG_IDS DEC DECIMAL DEFAULT DEFAULTS DEFINER %token DEALLOCATE DECLARE DEFERRABLE DEFERRED DELETE DELIMITER DEPENDS DESC DESTINATION DETACHED DETAILS %token DISABLE DISCARD DISTANCE DISTINCT DO DOMAIN DOUBLE DROP @@ -8840,10 +8840,6 @@ show_backup_options: { $$.val = &tree.ShowBackupOptions{EncryptionInfoDir: $3.expr()} } - | DEBUG_DUMP_METADATA_SST - { - $$.val = &tree.ShowBackupOptions{DebugMetadataSST: true} - } // %Help: SHOW CLUSTER SETTING - display cluster settings // %Category: Cfg @@ -18099,7 +18095,6 @@ unreserved_keyword: | DAY | DEALLOCATE | DEBUG_IDS -| DEBUG_DUMP_METADATA_SST | DECLARE | DELETE | DEFAULTS @@ -18624,7 +18619,6 @@ bare_label_keywords: | DATABASE | DATABASES | DEALLOCATE -| DEBUG_DUMP_METADATA_SST | DEBUG_IDS | DEC | DECIMAL diff --git a/pkg/sql/parser/testdata/show b/pkg/sql/parser/testdata/show index 3f28f8e79fe3..91f399f7f0d7 100644 --- a/pkg/sql/parser/testdata/show +++ b/pkg/sql/parser/testdata/show @@ -2141,13 +2141,13 @@ SHOW VIRTUAL CLUSTER foo WITH REPLICATION STATUS, PRIOR REPLICATION DETAILS, CAP SHOW VIRTUAL CLUSTER _ WITH REPLICATION STATUS, PRIOR REPLICATION DETAILS, CAPABILITIES -- identifiers removed parse -SHOW BACKUP 'family' IN ('string', 'placeholder', 'placeholder', 'placeholder', 'string', 'placeholder', 'string', 'placeholder') WITH incremental_location = 'nullif', privileges, debug_dump_metadata_sst +SHOW BACKUP 'family' IN ('string', 'placeholder', 'placeholder', 'placeholder', 'string', 'placeholder', 'string', 'placeholder') WITH incremental_location = 'nullif', privileges ---- -SHOW BACKUP 'family' IN ('*****', '*****', '*****', '*****', '*****', '*****', '*****', '*****') WITH OPTIONS (incremental_location = '*****', privileges, debug_dump_metadata_sst) -- normalized! -SHOW BACKUP ('family') IN (('*****'), ('*****'), ('*****'), ('*****'), ('*****'), ('*****'), ('*****'), ('*****')) WITH OPTIONS (incremental_location = ('*****'), privileges, debug_dump_metadata_sst) -- fully parenthesized -SHOW BACKUP '_' IN ('_', '_', '_', '_', '_', '_', '_', '_') WITH OPTIONS (incremental_location = '_', privileges, debug_dump_metadata_sst) -- literals removed -SHOW BACKUP 'family' IN ('*****', '*****', '*****', '*****', '*****', '*****', '*****', '*****') WITH OPTIONS (incremental_location = '*****', privileges, debug_dump_metadata_sst) -- identifiers removed -SHOW BACKUP 'family' IN ('string', 'placeholder', 'placeholder', 'placeholder', 'string', 'placeholder', 'string', 'placeholder') WITH OPTIONS (incremental_location = 'nullif', privileges, debug_dump_metadata_sst) -- passwords exposed +SHOW BACKUP 'family' IN ('*****', '*****', '*****', '*****', '*****', '*****', '*****', '*****') WITH OPTIONS (incremental_location = '*****', privileges) -- normalized! +SHOW BACKUP ('family') IN (('*****'), ('*****'), ('*****'), ('*****'), ('*****'), ('*****'), ('*****'), ('*****')) WITH OPTIONS (incremental_location = ('*****'), privileges) -- fully parenthesized +SHOW BACKUP '_' IN ('_', '_', '_', '_', '_', '_', '_', '_') WITH OPTIONS (incremental_location = '_', privileges) -- literals removed +SHOW BACKUP 'family' IN ('*****', '*****', '*****', '*****', '*****', '*****', '*****', '*****') WITH OPTIONS (incremental_location = '*****', privileges) -- identifiers removed +SHOW BACKUP 'family' IN ('string', 'placeholder', 'placeholder', 'placeholder', 'string', 'placeholder', 'string', 'placeholder') WITH OPTIONS (incremental_location = 'nullif', privileges) -- passwords exposed parse SHOW BACKUP 'abc' IN 'def' WITH SKIP SIZE diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index a5d0fee5ee62..8fb5797dc7f3 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -157,7 +157,6 @@ type ShowBackupOptions struct { // `ENCRYPTION-INFO` file necessary to decode the incremental backup lives in // the full backup dir. EncryptionInfoDir Expr - DebugMetadataSST bool CheckConnectionTransferSize Expr CheckConnectionDuration Expr @@ -220,10 +219,6 @@ func (o *ShowBackupOptions) Format(ctx *FmtCtx) { maybeAddSep() ctx.WriteString("skip size") } - if o.DebugMetadataSST { - maybeAddSep() - ctx.WriteString("debug_dump_metadata_sst") - } // The following are only used in connection-check SHOW. if o.CheckConnectionConcurrency != nil { @@ -253,7 +248,6 @@ func (o ShowBackupOptions) IsDefault() bool { o.EncryptionPassphrase == options.EncryptionPassphrase && o.Privileges == options.Privileges && o.SkipSize == options.SkipSize && - o.DebugMetadataSST == options.DebugMetadataSST && o.EncryptionInfoDir == options.EncryptionInfoDir && o.CheckConnectionTransferSize == options.CheckConnectionTransferSize && o.CheckConnectionDuration == options.CheckConnectionDuration && @@ -326,11 +320,6 @@ func (o *ShowBackupOptions) CombineWith(other *ShowBackupOptions) error { if err != nil { return err } - o.DebugMetadataSST, err = combineBools(o.DebugMetadataSST, other.DebugMetadataSST, - "debug_dump_metadata_sst") - if err != nil { - return err - } o.EncryptionInfoDir, err = combineExpr(o.EncryptionInfoDir, other.EncryptionInfoDir, "encryption_info_dir") if err != nil { From 8cdb3b57a13e31346fef92d80c46b4e2e1446956 Mon Sep 17 00:00:00 2001 From: David Taylor Date: Tue, 14 Jan 2025 21:02:52 +0000 Subject: [PATCH 125/126] jobs: remove legacy error log Release note: none. Epic: none. --- pkg/jobs/BUILD.bazel | 2 - pkg/jobs/config.go | 43 +-- pkg/jobs/errors.go | 25 -- pkg/jobs/helpers_test.go | 22 +- pkg/jobs/jobs.go | 90 ------ pkg/jobs/jobspb/jobs.proto | 7 +- pkg/jobs/registry.go | 23 +- pkg/jobs/registry_external_test.go | 297 ------------------ pkg/server/admin.go | 17 - pkg/server/application_api/jobs_test.go | 24 +- pkg/settings/registry.go | 2 + pkg/sql/crdb_internal.go | 17 - .../scdeps/sctestdeps/test_deps.go | 27 +- pkg/sql/schemachanger/schemachanger_test.go | 9 +- 14 files changed, 45 insertions(+), 560 deletions(-) diff --git a/pkg/jobs/BUILD.bazel b/pkg/jobs/BUILD.bazel index afeeba9a9a48..49025fc7df8a 100644 --- a/pkg/jobs/BUILD.bazel +++ b/pkg/jobs/BUILD.bazel @@ -64,7 +64,6 @@ go_library( "//pkg/util/cidr", "//pkg/util/envutil", "//pkg/util/hlc", - "//pkg/util/json", "//pkg/util/log", "//pkg/util/log/eventpb", "//pkg/util/log/severity", @@ -83,7 +82,6 @@ go_library( "@com_github_cockroachdb_errors//oserror", "@com_github_cockroachdb_logtags//:logtags", "@com_github_cockroachdb_redact//:redact", - "@com_github_gogo_protobuf//jsonpb", "@com_github_gogo_protobuf//proto", "@com_github_gogo_protobuf//types", "@com_github_klauspost_compress//gzip", diff --git a/pkg/jobs/config.go b/pkg/jobs/config.go index 2df950884cb1..261e0b5f12d0 100644 --- a/pkg/jobs/config.go +++ b/pkg/jobs/config.go @@ -16,16 +16,14 @@ import ( ) const ( - intervalBaseSettingKey = "jobs.registry.interval.base" - adoptIntervalSettingKey = "jobs.registry.interval.adopt" - cancelIntervalSettingKey = "jobs.registry.interval.cancel" - gcIntervalSettingKey = "jobs.registry.interval.gc" - retentionTimeSettingKey = "jobs.retention_time" - cancelUpdateLimitKey = "jobs.cancel_update_limit" - executionErrorsMaxEntriesKey = "jobs.execution_errors.max_entries" - executionErrorsMaxEntrySizeKey = "jobs.execution_errors.max_entry_size" - debugPausePointsSettingKey = "jobs.debug.pausepoints" - metricsPollingIntervalKey = "jobs.metrics.interval.poll" + intervalBaseSettingKey = "jobs.registry.interval.base" + adoptIntervalSettingKey = "jobs.registry.interval.adopt" + cancelIntervalSettingKey = "jobs.registry.interval.cancel" + gcIntervalSettingKey = "jobs.registry.interval.gc" + retentionTimeSettingKey = "jobs.retention_time" + cancelUpdateLimitKey = "jobs.cancel_update_limit" + debugPausePointsSettingKey = "jobs.debug.pausepoints" + metricsPollingIntervalKey = "jobs.metrics.interval.poll" ) const ( @@ -49,15 +47,6 @@ const ( // updated when canceling jobs concurrently from dead sessions. defaultCancellationsUpdateLimit int64 = 1000 - // defaultExecutionErrorsMaxEntries is the default number of error entries - // which will be retained. - defaultExecutionErrorsMaxEntries = 3 - - // defaultExecutionErrorsMaxEntrySize is the maximum allowed size of an - // error. If this size is exceeded, the error will be formatted as a string - // and then truncated to fit the size. - defaultExecutionErrorsMaxEntrySize = 64 << 10 // 64 KiB - // defaultPollForMetricsInterval is the default interval to poll the jobs // table for metrics. defaultPollForMetricsInterval = 30 * time.Second @@ -127,22 +116,6 @@ var ( settings.NonNegativeInt, ) - executionErrorsMaxEntriesSetting = settings.RegisterIntSetting( - settings.ApplicationLevel, - executionErrorsMaxEntriesKey, - "the maximum number of retriable error entries which will be stored for introspection", - defaultExecutionErrorsMaxEntries, - settings.NonNegativeInt, - ) - - executionErrorsMaxEntrySize = settings.RegisterByteSizeSetting( - settings.ApplicationLevel, - executionErrorsMaxEntrySizeKey, - "the maximum byte size of individual error entries which will be stored"+ - " for introspection", - defaultExecutionErrorsMaxEntrySize, - ) - debugPausepoints = settings.RegisterStringSetting( settings.ApplicationLevel, debugPausePointsSettingKey, diff --git a/pkg/jobs/errors.go b/pkg/jobs/errors.go index acba73669233..4da9c046b6d1 100644 --- a/pkg/jobs/errors.go +++ b/pkg/jobs/errors.go @@ -6,14 +6,12 @@ package jobs import ( - "context" "fmt" "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) @@ -163,26 +161,3 @@ func (e *retriableExecutionError) SafeFormatError(p errors.Printer) error { } return e.cause } - -func (e *retriableExecutionError) toRetriableExecutionFailure( - ctx context.Context, maxErrorSize int, -) *jobspb.RetriableExecutionFailure { - // If the cause is too large, we format it, losing all structure, and retain - // a prefix. - ef := &jobspb.RetriableExecutionFailure{ - Status: string(e.status), - ExecutionStartMicros: timeutil.ToUnixMicros(e.start), - ExecutionEndMicros: timeutil.ToUnixMicros(e.end), - InstanceID: e.instanceID, - } - if encodedCause := errors.EncodeError(ctx, e.cause); encodedCause.Size() < maxErrorSize { - ef.Error = &encodedCause - } else { - formatted := e.cause.Error() - if len(formatted) > maxErrorSize { - formatted = formatted[:maxErrorSize] - } - ef.TruncatedError = formatted - } - return ef -} diff --git a/pkg/jobs/helpers_test.go b/pkg/jobs/helpers_test.go index 2ceae13716a6..7b883b99e899 100644 --- a/pkg/jobs/helpers_test.go +++ b/pkg/jobs/helpers_test.go @@ -83,18 +83,16 @@ func (j *Job) TestingCurrentStatus(ctx context.Context) (Status, error) { } const ( - AdoptQuery = claimQuery - CancelQuery = pauseAndCancelUpdate - RemoveClaimsQuery = removeClaimsForDeadSessionsQuery - ProcessJobsQuery = processQuery - IntervalBaseSettingKey = intervalBaseSettingKey - AdoptIntervalSettingKey = adoptIntervalSettingKey - CancelIntervalSettingKey = cancelIntervalSettingKey - GcIntervalSettingKey = gcIntervalSettingKey - RetentionTimeSettingKey = retentionTimeSettingKey - DefaultAdoptInterval = defaultAdoptInterval - ExecutionErrorsMaxEntriesKey = executionErrorsMaxEntriesKey - ExecutionErrorsMaxEntrySizeKey = executionErrorsMaxEntrySizeKey + AdoptQuery = claimQuery + CancelQuery = pauseAndCancelUpdate + RemoveClaimsQuery = removeClaimsForDeadSessionsQuery + ProcessJobsQuery = processQuery + IntervalBaseSettingKey = intervalBaseSettingKey + AdoptIntervalSettingKey = adoptIntervalSettingKey + CancelIntervalSettingKey = cancelIntervalSettingKey + GcIntervalSettingKey = gcIntervalSettingKey + RetentionTimeSettingKey = retentionTimeSettingKey + DefaultAdoptInterval = defaultAdoptInterval ) var ( diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go index f9569c5b75d6..3352a962d448 100644 --- a/pkg/jobs/jobs.go +++ b/pkg/jobs/jobs.go @@ -6,9 +6,7 @@ package jobs import ( - "bytes" "context" - gojson "encoding/json" "fmt" "reflect" "sync/atomic" @@ -21,13 +19,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" - "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" - "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -36,7 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" - "github.com/gogo/protobuf/jsonpb" ) // Job manages logging the progress of long-running system processes, like @@ -892,90 +886,6 @@ func (sj *StartableJob) recordStart() (alreadyStarted bool) { return atomic.AddInt64(&sj.starts, 1) != 1 } -// ParseRetriableExecutionErrorLogFromJSON inverts the output of -// FormatRetriableExecutionErrorLogToJSON. -func ParseRetriableExecutionErrorLogFromJSON( - log []byte, -) ([]*jobspb.RetriableExecutionFailure, error) { - var jsonArr []gojson.RawMessage - if err := gojson.Unmarshal(log, &jsonArr); err != nil { - return nil, errors.Wrap(err, "failed to decode json array for execution log") - } - ret := make([]*jobspb.RetriableExecutionFailure, len(jsonArr)) - - json := jsonpb.Unmarshaler{AllowUnknownFields: true} - var reader bytes.Reader - for i, data := range jsonArr { - msgI, err := protoreflect.NewMessage("cockroach.sql.jobs.jobspb.RetriableExecutionFailure") - if err != nil { - return nil, errors.WithAssertionFailure(err) - } - msg := msgI.(*jobspb.RetriableExecutionFailure) - reader.Reset(data) - if err := json.Unmarshal(&reader, msg); err != nil { - return nil, err - } - ret[i] = msg - } - return ret, nil -} - -// FormatRetriableExecutionErrorLogToJSON extracts the events -// stored in the payload, formats them into a json array. This function -// is intended for use with crdb_internal.jobs. Note that the error will -// be flattened into a string and stored in the TruncatedError field. -func FormatRetriableExecutionErrorLogToJSON( - ctx context.Context, log []*jobspb.RetriableExecutionFailure, -) (*tree.DJSON, error) { - ab := json.NewArrayBuilder(len(log)) - for i := range log { - ev := *log[i] - if ev.Error != nil { - ev.TruncatedError = errors.DecodeError(ctx, *ev.Error).Error() - ev.Error = nil - } - msg, err := protoreflect.MessageToJSON(&ev, protoreflect.FmtFlags{ - EmitDefaults: false, - }) - if err != nil { - return nil, err - } - ab.Add(msg) - } - return tree.NewDJSON(ab.Build()), nil -} - -// FormatRetriableExecutionErrorLogToStringArray extracts the events -// stored in the payload, formats them into strings and returns them as an -// array of strings. This function is intended for use with crdb_internal.jobs. -func FormatRetriableExecutionErrorLogToStringArray( - ctx context.Context, log []*jobspb.RetriableExecutionFailure, -) *tree.DArray { - arr := tree.NewDArray(types.String) - for _, ev := range log { - if ev == nil { // no reason this should happen, but be defensive - continue - } - var cause error - if ev.Error != nil { - cause = errors.DecodeError(ctx, *ev.Error) - } else { - cause = fmt.Errorf("(truncated) %s", ev.TruncatedError) - } - msg := formatRetriableExecutionFailure( - ev.InstanceID, - Status(ev.Status), - timeutil.FromUnixMicros(ev.ExecutionStartMicros), - timeutil.FromUnixMicros(ev.ExecutionEndMicros), - cause, - ) - // We really don't care about errors here. I'd much rather see nothing - // in my log than crash. - _ = arr.Append(tree.NewDString(msg)) - } - return arr -} - // GetJobTraceID returns the current trace ID of the job from the job progress. func GetJobTraceID(ctx context.Context, db isql.DB, jobID jobspb.JobID) (tracingpb.TraceID, error) { var traceID tracingpb.TraceID diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 9ae417920c00..f4092fc4ba39 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -1498,12 +1498,9 @@ message Payload { // PauseReason is used to describe the reason that the job is currently paused // or has been requested to be paused. string pause_reason = 28; - // RetriableExecutionFailureLog stores a history of retriable execution - // failures. These failures may occur in either the RUNNING or REVERTING - // status. A finite number of these entries will be kept, as governed by - // the jobs.execution_errors.max_entries cluster setting. - repeated RetriableExecutionFailure retriable_execution_failure_log = 32; + reserved 32; + // CreationClusterID is populated at creation with the ClusterID, in case a // job resuming later, needs to use this information, e.g. to determine if it // has been restored into a different cluster, which might mean it should diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index 6b77540a20dc..74899bece55c 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -1983,25 +1983,12 @@ func (r *Registry) maybeRecordExecutionFailure(ctx context.Context, err error, j if !errors.As(err, &efe) { return } - - updateErr := j.NoTxn().Update(ctx, func( - txn isql.Txn, md JobMetadata, ju *JobUpdater, - ) error { - pl := md.Payload - { // Append the entry to the log - maxSize := int(executionErrorsMaxEntrySize.Get(&r.settings.SV)) - pl.RetriableExecutionFailureLog = append(pl.RetriableExecutionFailureLog, - efe.toRetriableExecutionFailure(ctx, maxSize)) - } - { // Maybe truncate the log. - maxEntries := int(executionErrorsMaxEntriesSetting.Get(&r.settings.SV)) - log := &pl.RetriableExecutionFailureLog - if len(*log) > maxEntries { - *log = (*log)[len(*log)-maxEntries:] - } + updateErr := r.db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + v, err := txn.GetSystemSchemaVersion(ctx) + if err != nil || v.Less(clusterversion.V25_1.Version()) { + return err } - ju.UpdatePayload(pl) - return nil + return j.Messages().Record(ctx, txn, "retry", efe.cause.Error()) }) if ctx.Err() != nil { return diff --git a/pkg/jobs/registry_external_test.go b/pkg/jobs/registry_external_test.go index a7b0e5bdd0b2..3f60a9220987 100644 --- a/pkg/jobs/registry_external_test.go +++ b/pkg/jobs/registry_external_test.go @@ -12,7 +12,6 @@ import ( "fmt" "reflect" "regexp" - "strconv" "strings" "sync/atomic" "testing" @@ -29,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/isql" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slstorage" @@ -39,7 +37,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" @@ -417,300 +414,6 @@ func TestGCDurationControl(t *testing.T) { tdb.CheckQueryResultsRetry(t, existsQuery, [][]string{{"false"}}) } -// TestErrorsPopulatedOnRetry confirms that when a job fails with a retriable -// error that that error makes its way to the payload. -func TestErrorsPopulatedOnRetry(t *testing.T) { - defer leaktest.AfterTest(t)() - ls := log.Scope(t) - defer ls.Close(t) - - type event struct { - id jobspb.JobID - resume chan error - } - mkEvent := func(j *jobs.Job) event { - return event{id: j.ID(), resume: make(chan error)} - } - evChan := make(chan event) - defer jobs.TestingRegisterConstructor(jobspb.TypeImport, func(j *jobs.Job, cs *cluster.Settings) jobs.Resumer { - execFn := func(ctx context.Context) error { - ev := mkEvent(j) - select { - case evChan <- ev: - case <-ctx.Done(): - return ctx.Err() - } - select { - case err := <-ev.resume: - return err - case <-ctx.Done(): - return ctx.Err() - } - } - return jobstest.FakeResumer{ - OnResume: execFn, - FailOrCancel: execFn, - } - }, jobs.UsesTenantCostControl)() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ - Knobs: base.TestingKnobs{ - JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), - }, - }) - tdb := sqlutils.MakeSQLRunner(sqlDB) - ctx := context.Background() - defer s.Stopper().Stop(ctx) - registry := s.JobRegistry().(*jobs.Registry) - mkJob := func(t *testing.T) jobspb.JobID { - id := registry.MakeJobID() - _, err := registry.CreateJobWithTxn(ctx, jobs.Record{ - // Job does not accept an empty Details field, so arbitrarily provide - // ImportDetails. - Details: jobspb.ImportDetails{}, - Progress: jobspb.ImportProgress{}, - Username: username.TestUserName(), - }, id, nil /* txn */) - require.NoError(t, err) - return id - } - type parsedError struct { - start, end time.Time - status jobs.Status - error string - instance base.SQLInstanceID - } - var ( - executionErrorRE = regexp.MustCompile( - `(?P\w+) execution from '(?P.*)' to '(?P.*)' on (?P\d+) failed: (?P.*)`, - ) - statusIdx = executionErrorRE.SubexpIndex("status") - startIdx = executionErrorRE.SubexpIndex("Start") - endIdx = executionErrorRE.SubexpIndex("End") - instanceIdx = executionErrorRE.SubexpIndex("instance") - errorIdx = executionErrorRE.SubexpIndex("error") - ) - parseTimestamp := func(t *testing.T, s string) time.Time { - ptc := tree.NewParseContext(timeutil.Now()) - ts, _, err := tree.ParseDTimestamp(ptc, s, time.Microsecond) - require.NoError(t, err) - return ts.Time - } - parseInstanceID := func(t *testing.T, s string) base.SQLInstanceID { - i, err := strconv.ParseInt(s, 10, 32) - require.NoError(t, err) - return base.SQLInstanceID(i) - } - parseExecutionError := func(t *testing.T, s string) (ret parsedError) { - matches := executionErrorRE.FindStringSubmatch(s) - require.NotNil(t, matches) - ret.status = jobs.Status(matches[statusIdx]) - ret.start = parseTimestamp(t, matches[startIdx]) - ret.end = parseTimestamp(t, matches[endIdx]) - ret.instance = parseInstanceID(t, matches[instanceIdx]) - ret.error = matches[errorIdx] - return ret - } - parseExecutionErrors := func(t *testing.T, s [][]string) (ret []parsedError) { - for _, res := range s { - require.Len(t, res, 1) - ret = append(ret, parseExecutionError(t, res[0])) - } - return ret - } - executionErrorEqual := func(t *testing.T, a, b parsedError) { - require.Equal(t, a.instance, b.instance) - require.Equal(t, a.error, b.error) - require.Equal(t, a.status, b.status) - } - waitForEvent := func(t *testing.T, id jobspb.JobID) (ev event, start time.Time) { - ev = <-evChan - require.Equal(t, id, ev.id) - tdb.QueryRow(t, "SELECT now() FROM crdb_internal.jobs WHERE job_id = $1", id).Scan(&start) - return ev, start - } - checkExecutionError := func( - t *testing.T, execErr parsedError, status jobs.Status, _, _ time.Time, cause string, - ) { - require.Equal(t, base.SQLInstanceID(1), execErr.instance) - require.Equal(t, status, execErr.status) - require.Equal(t, cause, execErr.error) - } - getExecErrors := func(t *testing.T, id jobspb.JobID) []parsedError { - return parseExecutionErrors(t, - tdb.QueryStr(t, ` -SELECT unnest(execution_errors) - FROM crdb_internal.jobs - WHERE job_id = $1;`, id), - ) - } - checkLogEntry := func( - t *testing.T, id jobspb.JobID, status jobs.Status, - from, to time.Time, cause string, - ) { - log.FlushFiles() - entries, err := log.FetchEntriesFromFiles( - from.UnixNano(), to.UnixNano(), 2, - regexp.MustCompile(fmt.Sprintf( - "job %d: %s execution encountered retriable error: %s", - id, status, cause, - )), - log.WithFlattenedSensitiveData, - ) - require.NoError(t, err) - require.Len(t, entries, 1) - } - t.Run("retriable error makes it into payload", func(t *testing.T) { - id := mkJob(t) - firstRun, firstStart := waitForEvent(t, id) - const err1 = "boom1" - firstRun.resume <- jobs.MarkAsRetryJobError(errors.New(err1)) - - // Wait for the job to get restarted. - secondRun, secondStart := waitForEvent(t, id) - - // Confirm the previous execution error was properly recorded. - var firstExecErr parsedError - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 1) - firstExecErr = execErrs[0] - checkExecutionError(t, firstExecErr, jobs.StatusRunning, firstStart, secondStart, err1) - checkLogEntry(t, id, jobs.StatusRunning, firstStart, secondStart, err1) - } - const err2 = "boom2" - secondRun.resume <- jobs.MarkAsRetryJobError(errors.New(err2)) - thirdRun, thirdStart := waitForEvent(t, id) - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 2) - executionErrorEqual(t, firstExecErr, execErrs[0]) - secondExecErr := execErrs[1] - checkExecutionError(t, secondExecErr, jobs.StatusRunning, secondStart, thirdStart, err2) - checkLogEntry(t, id, jobs.StatusRunning, secondStart, thirdStart, err2) - } - close(thirdRun.resume) - require.NoError(t, registry.WaitForJobs(ctx, []jobspb.JobID{id})) - }) - t.Run("fail or cancel error", func(t *testing.T) { - id := mkJob(t) - firstRun, firstStart := waitForEvent(t, id) - const err1 = "boom1" - firstRun.resume <- jobs.MarkAsRetryJobError(errors.New(err1)) - - // Wait for the job to get restarted. - secondRun, secondStart := waitForEvent(t, id) - - // Confirm the previous execution error was properly recorded. - var firstExecErr parsedError - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 1) - firstExecErr = execErrs[0] - checkExecutionError(t, firstExecErr, jobs.StatusRunning, firstStart, secondStart, err1) - checkLogEntry(t, id, jobs.StatusRunning, firstStart, secondStart, err1) - } - const err2 = "boom2" - secondRun.resume <- errors.New(err2) - thirdRun, thirdStart := waitForEvent(t, id) // thirdRun is Reverting - // Confirm that no new error was recorded in the log. It will be in - // FinalResumeError. - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 1) - executionErrorEqual(t, firstExecErr, execErrs[0]) - } - const err3 = "boom3" - thirdRun.resume <- jobs.MarkAsRetryJobError(errors.New(err3)) - fourthRun, fourthStart := waitForEvent(t, id) - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 2) - executionErrorEqual(t, firstExecErr, execErrs[0]) - checkExecutionError(t, execErrs[1], jobs.StatusReverting, thirdStart, fourthStart, err3) - checkLogEntry(t, id, jobs.StatusReverting, thirdStart, fourthStart, err3) - } - close(fourthRun.resume) - require.Regexp(t, err2, registry.WaitForJobs(ctx, []jobspb.JobID{id})) - }) - t.Run("truncation", func(t *testing.T) { - id := mkJob(t) - firstRun, firstStart := waitForEvent(t, id) - const maxSize, largeSize = 2 << 10, 8 << 10 - tdb.Exec(t, "SET CLUSTER SETTING "+jobs.ExecutionErrorsMaxEntrySizeKey+" = $1", maxSize) - tdb.Exec(t, "SET CLUSTER SETTING "+jobs.ExecutionErrorsMaxEntriesKey+" = $1", 1) - err1 := strings.Repeat("a", largeSize) - firstRun.resume <- jobs.MarkAsRetryJobError(fmt.Errorf("%s", err1)) - - // Wait for the job to get restarted. - secondRun, secondStart := waitForEvent(t, id) - // Confirm the previous execution error was properly recorded. - var firstExecErr parsedError - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 1) - firstExecErr = execErrs[0] - // Ensure we see the truncated error in the table but the full error - // in the logs. - expTruncatedError := "(truncated) " + err1[:maxSize] - checkExecutionError(t, firstExecErr, jobs.StatusRunning, firstStart, secondStart, expTruncatedError) - checkLogEntry(t, id, jobs.StatusRunning, firstStart, secondStart, err1) - } - const err2 = "boom2" - secondRun.resume <- jobs.MarkAsRetryJobError(errors.New(err2)) - thirdRun, thirdStart := waitForEvent(t, id) - var secondExecErr parsedError - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 1) - secondExecErr = execErrs[0] - checkExecutionError(t, secondExecErr, jobs.StatusRunning, secondStart, thirdStart, err2) - checkLogEntry(t, id, jobs.StatusRunning, secondStart, thirdStart, err2) - } - // Fail the job so we can also test the truncation of reverting retry - // errors. - const err3 = "boom3" - thirdRun.resume <- errors.New(err3) // not retriable - fourthRun, fourthStart := waitForEvent(t, id) // first Reverting run - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 1) - executionErrorEqual(t, secondExecErr, execErrs[0]) - } - err4 := strings.Repeat("b", largeSize) - fourthRun.resume <- jobs.MarkAsRetryJobError(fmt.Errorf("%s", err4)) - fifthRun, fifthStart := waitForEvent(t, id) - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 1) - // Ensure we see the truncated error in the table but the full error - // in the logs. - expTruncatedError := "(truncated) " + err4[:maxSize] - checkExecutionError(t, execErrs[0], jobs.StatusReverting, fourthStart, fifthStart, expTruncatedError) - checkLogEntry(t, id, jobs.StatusReverting, fourthStart, fifthStart, err4) - } - const err5 = "boom5" - fifthRun.resume <- jobs.MarkAsRetryJobError(errors.New(err5)) - sixthRun, sixthStart := waitForEvent(t, id) - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 1) - checkExecutionError(t, execErrs[0], jobs.StatusReverting, fifthStart, sixthStart, err5) - checkLogEntry(t, id, jobs.StatusReverting, fifthStart, sixthStart, err5) - } - const err6 = "boom5" - tdb.Exec(t, "SET CLUSTER SETTING "+jobs.ExecutionErrorsMaxEntriesKey+" = $1", 0) - sixthRun.resume <- jobs.MarkAsRetryJobError(errors.New(err6)) - seventhRun, seventhStart := waitForEvent(t, id) - { - execErrs := getExecErrors(t, id) - require.Len(t, execErrs, 0) - checkLogEntry(t, id, jobs.StatusReverting, sixthStart, seventhStart, err6) - } - close(seventhRun.resume) - require.Regexp(t, err3, registry.WaitForJobs(ctx, []jobspb.JobID{id})) - }) -} - // TestWaitWithRetryableError tests retryable errors when querying // for jobs. func TestWaitWithRetryableError(t *testing.T) { diff --git a/pkg/server/admin.go b/pkg/server/admin.go index cc7d3c556889..4e48eb82153f 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -2396,23 +2396,6 @@ func scanRowIntoJob(scanner resultScanner, row tree.Datums, job *serverpb.JobRes if runningStatusOrNil != nil { job.RunningStatus = *runningStatusOrNil } - if executionFailuresOrNil != nil { - failures, err := jobs.ParseRetriableExecutionErrorLogFromJSON([]byte(*executionFailuresOrNil)) - if err != nil { - return errors.Wrap(err, "parse") - } - job.ExecutionFailures = make([]*serverpb.JobResponse_ExecutionFailure, len(failures)) - for i, f := range failures { - start := time.UnixMicro(f.ExecutionStartMicros) - end := time.UnixMicro(f.ExecutionEndMicros) - job.ExecutionFailures[i] = &serverpb.JobResponse_ExecutionFailure{ - Status: f.Status, - Start: &start, - End: &end, - Error: f.TruncatedError, - } - } - } if coordinatorOrNil != nil { job.CoordinatorID = *coordinatorOrNil } diff --git a/pkg/server/application_api/jobs_test.go b/pkg/server/application_api/jobs_test.go index 621b91bd89ce..35de5200433d 100644 --- a/pkg/server/application_api/jobs_test.go +++ b/pkg/server/application_api/jobs_test.go @@ -146,9 +146,8 @@ func TestAdminAPIJobs(t *testing.T) { } for _, job := range testJobs { payload := jobspb.Payload{ - UsernameProto: job.username.EncodeProto(), - Details: jobspb.WrapPayloadDetails(job.details), - RetriableExecutionFailureLog: job.executionFailures, + UsernameProto: job.username.EncodeProto(), + Details: jobspb.WrapPayloadDetails(job.details), } payloadBytes, err := protoutil.Marshal(&payload) if err != nil { @@ -337,9 +336,8 @@ func TestAdminAPIJobsDetails(t *testing.T) { } for _, job := range testJobs { payload := jobspb.Payload{ - UsernameProto: job.username.EncodeProto(), - Details: jobspb.WrapPayloadDetails(job.details), - RetriableExecutionFailureLog: job.executionLog, + UsernameProto: job.username.EncodeProto(), + Details: jobspb.WrapPayloadDetails(job.details), } payloadBytes, err := protoutil.Marshal(&payload) if err != nil { @@ -391,20 +389,6 @@ func TestAdminAPIJobsDetails(t *testing.T) { for i, job := range resJobs { require.Equal(t, testJobs[i].id, job.ID) - require.Equal(t, len(testJobs[i].executionLog), len(job.ExecutionFailures)) - for j, f := range job.ExecutionFailures { - tf := testJobs[i].executionLog[j] - require.Equal(t, tf.Status, f.Status) - require.Equal(t, tf.ExecutionStartMicros, f.Start.UnixMicro()) - require.Equal(t, tf.ExecutionEndMicros, f.End.UnixMicro()) - var expErr string - if tf.Error != nil { - expErr = errors.DecodeError(context.Background(), *tf.Error).Error() - } else { - expErr = tf.TruncatedError - } - require.Equal(t, expErr, f.Error) - } } } diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index b4b9fce7f15e..6f541084eb87 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -252,6 +252,8 @@ var retiredSettings = map[InternalKey]struct{}{ "storage.single_delete.crash_on_ineffectual.enabled": {}, "bulkio.backup.elide_common_prefix.enabled": {}, "kv.bulkio.write_metadata_sst.enabled": {}, + "jobs.execution_errors.max_entries": {}, + "jobs.execution_errors.max_entry_size": {}, } // grandfatheredDefaultSettings is the list of "grandfathered" existing sql.defaults diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 0b45ebeff0ee..52840ff1c7cc 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1365,23 +1365,6 @@ func makeJobsTableRows( traceID = tree.NewDInt(tree.DInt(progress.TraceID)) } } - if len(payload.RetriableExecutionFailureLog) > 0 { - executionErrors = jobs.FormatRetriableExecutionErrorLogToStringArray( - ctx, payload.RetriableExecutionFailureLog, - ) - // It's not clear why we'd ever see an error here, - var err error - executionEvents, err = jobs.FormatRetriableExecutionErrorLogToJSON( - ctx, payload.RetriableExecutionFailureLog, - ) - if err != nil { - if errorStr == tree.DNull { - errorStr = tree.NewDString(errors.Wrap(err, "failed to marshal execution error log").Error()) - } else { - executionEvents = tree.DNull - } - } - } if err = addRow( id, diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index 0ddcd7db25dd..3a44596659f8 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -1112,20 +1112,19 @@ func (s *TestState) UpdateSchemaChangeJob( TraceID: 0, } oldPayload := jobspb.Payload{ - Description: scJob.Description, - Statement: scJob.Statements, - UsernameProto: scJob.Username.EncodeProto(), - StartedMicros: 0, - FinishedMicros: 0, - DescriptorIDs: scJob.DescriptorIDs, - Error: "", - ResumeErrors: nil, - CleanupErrors: nil, - FinalResumeError: nil, - Noncancelable: scJob.NonCancelable, - Details: jobspb.WrapPayloadDetails(scJob.Details), - PauseReason: "", - RetriableExecutionFailureLog: nil, + Description: scJob.Description, + Statement: scJob.Statements, + UsernameProto: scJob.Username.EncodeProto(), + StartedMicros: 0, + FinishedMicros: 0, + DescriptorIDs: scJob.DescriptorIDs, + Error: "", + ResumeErrors: nil, + CleanupErrors: nil, + FinalResumeError: nil, + Noncancelable: scJob.NonCancelable, + Details: jobspb.WrapPayloadDetails(scJob.Details), + PauseReason: "", } oldJobMetadata := jobs.JobMetadata{ ID: scJob.JobID, diff --git a/pkg/sql/schemachanger/schemachanger_test.go b/pkg/sql/schemachanger/schemachanger_test.go index 01454eedaa68..7e949933fd3c 100644 --- a/pkg/sql/schemachanger/schemachanger_test.go +++ b/pkg/sql/schemachanger/schemachanger_test.go @@ -129,11 +129,6 @@ func TestSchemaChangerJobErrorDetails(t *testing.T) { tdb.ExpectErr(t, `boom`, `ALTER TABLE db.t ADD COLUMN b INT NOT NULL DEFAULT (123)`) jobID := jobspb.JobID(atomic.LoadInt64(&jobIDValue)) - // Check that the error is featured in the jobs table. - results := tdb.QueryStr(t, `SELECT execution_errors FROM crdb_internal.jobs WHERE job_id = $1`, jobID) - require.Len(t, results, 1) - require.Regexp(t, `^\{\"reverting execution from .* on 1 failed: boom\"\}$`, results[0][0]) - // Check that the error details are also featured in the jobs table. checkErrWithDetails := func(ee *errorspb.EncodedError) { require.NotNil(t, ee) @@ -146,7 +141,7 @@ func TestSchemaChangerJobErrorDetails(t *testing.T) { require.Regexp(t, "^stages graphviz: https.*", ed[1]) require.Regexp(t, "^dependencies graphviz: https.*", ed[2]) } - results = tdb.QueryStr(t, `SELECT encode(payload, 'hex') FROM crdb_internal.system_jobs WHERE id = $1`, jobID) + results := tdb.QueryStr(t, `SELECT encode(payload, 'hex') FROM crdb_internal.system_jobs WHERE id = $1`, jobID) require.Len(t, results, 1) b, err := hex.DecodeString(results[0][0]) require.NoError(t, err) @@ -154,8 +149,6 @@ func TestSchemaChangerJobErrorDetails(t *testing.T) { err = protoutil.Unmarshal(b, &p) require.NoError(t, err) checkErrWithDetails(p.FinalResumeError) - require.LessOrEqual(t, 1, len(p.RetriableExecutionFailureLog)) - checkErrWithDetails(p.RetriableExecutionFailureLog[0].Error) // Check that the error is featured in the event log. const eventLogCountQuery = `SELECT count(*) FROM system.eventlog WHERE "eventType" = $1` From 3d460c1f5b9fee0cecea88d347150a7c93c5b8a3 Mon Sep 17 00:00:00 2001 From: Ibrahim Kettaneh Date: Wed, 15 Jan 2025 12:11:35 -0500 Subject: [PATCH 126/126] raft: ComputeLeadSupportUntil on every leader tick This commit calls ComputeLeadSupportUntil on every leader tick rather than on every leader step. If the range is idle, the leader won't have anything to step on, but it will keep ticking. References: #139072 Release Note: None --- pkg/raft/raft.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/raft/raft.go b/pkg/raft/raft.go index 83f1dec0af1e..6e94ac21dcc5 100644 --- a/pkg/raft/raft.go +++ b/pkg/raft/raft.go @@ -1210,6 +1210,9 @@ func (r *raft) tickElection() { func (r *raft) tickHeartbeat() { assertTrue(r.state == pb.StateLeader, "tickHeartbeat called by non-leader") + // Compute the LeadSupportUntil on every tick. + r.fortificationTracker.ComputeLeadSupportUntil(r.state) + // Check if we intended to step down. If so, step down if it's safe to do so. // Otherwise, continue doing leader things. if r.fortificationTracker.SteppingDown() && r.fortificationTracker.CanDefortify() { @@ -1841,9 +1844,6 @@ func (r *raft) logMsgHigherTerm(m pb.Message, suffix redact.SafeString) { type stepFunc func(r *raft, m pb.Message) error func stepLeader(r *raft, m pb.Message) error { - // Compute the LeadSupportUntil on every tick. - r.fortificationTracker.ComputeLeadSupportUntil(r.state) - // These message types do not require any progress for m.From. switch m.Type { case pb.MsgBeat:
changefeed.event_consumer_worker_queue_size
integer16if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of events which a worker can bufferServerless/Dedicated/Self-Hosted
changefeed.event_consumer_workers
integer0the number of workers to use when processing events: <0 disables, 0 assigns a reasonable default, >0 assigns the setting value. for experimental/core changefeeds and changefeeds using parquet format, this is disabledServerless/Dedicated/Self-Hosted
changefeed.fast_gzip.enabled
booleantrueuse fast gzip implementationServerless/Dedicated/Self-Hosted
changefeed.frontier_highwater_lag_checkpoint_threshold
duration10m0scontrols the maximum the high-water mark is allowed to lag behind the leading spans of the frontier before per-span checkpointing is enabled; if 0, checkpointing due to high-water lag is disabledServerless/Dedicated/Self-Hosted
changefeed.span_checkpoint.lag_threshold
(alias: changefeed.frontier_highwater_lag_checkpoint_threshold)
duration10m0sthe amount of time a changefeed's lagging (slowest) spans must lag behind its leading (fastest) spans before a span-level checkpoint to save leading span progress is written; if 0, span-level checkpoints due to lagging spans is disabledServerless/Dedicated/Self-Hosted
changefeed.memory.per_changefeed_limit
byte size512 MiBcontrols amount of data that can be buffered per changefeedServerless/Dedicated/Self-Hosted
changefeed.resolved_timestamp.min_update_interval
(alias: changefeed.min_highwater_advance)
duration0sminimum amount of time that must have elapsed since the last time a changefeed's resolved timestamp was updated before it is eligible to be updated again; default of 0 means no minimum interval is enforced but updating will still be limited by the average time it takes to checkpoint progressServerless/Dedicated/Self-Hosted
changefeed.node_throttle_config
stringspecifies node level throttling configuration for all changefeeedsServerless/Dedicated/Self-Hosted