Skip to content

Commit

Permalink
Merge #69047
Browse files Browse the repository at this point in the history
69047: server,roachpb: introduce span config rpcs r=irfansharif a=irfansharif

Part of #67679; these RPCs are exposed through the `kvtenant.Connector`
interface for tenants and also sit on `pkg/server.(*Node)` for the host
tenant. The basic type in these RPCs is the `SpanConfig` proto, which is
the same as our existing `ZoneConfig` proto but without any inheritance
business.

    // GetSpanConfigsRequest is used to fetch the span configurations
    // over the specified keyspans.
    message GetSpanConfigsRequest {
      // Spans to request the configurations for. The spans listed here
      // are not allowed to overlap with one another.
      repeated Span spans = 1 [(gogoproto.nullable) = false];
    };

    // GetSpanConfigsResponse lists out the span configurations that
    // overlap with the requested spans.
    message GetSpanConfigsResponse {
      // SpanConfigEntries capture the span configurations over the
      // requested spans.  The results for each Span in the matching
      // GetSpanConfigsRequest are flattened out into a single slice. It's
      // possible for there to be no configurations for a given span;
      // there'll simply be no entries for it.
      repeated SpanConfigEntry span_config_entries = 1 [(gogoproto.nullable) = false];
    };

    // UpdateSpanConfigsRequest is used to update the span
    // configurations over the given spans.
    //
    // This is a "targeted" API: the spans being deleted are expected to
    // have been present with the same bounds (same start/end key); the
    // same is true for spans being updated with new configs. If spans
    // are being added, they're expected to not overlap with any
    // existing spans. When divvying up an existing span into multiple
    // others, callers are expected to delete the old and upsert the new
    // ones. This can happen as part of the same request; we delete the
    // spans marked for deletion before upserting whatever was
    // requested.
    //
    // Spans are not allowed to overlap with other spans in the same
    // list but can across lists. This is necessary to support the
    // delete+upsert semantics described above.
    message UpdateSpanConfigsRequest {
      // ToDelete lists out the spans we want to delete span configs
      // for.
      repeated Span to_delete = 1 [(gogoproto.nullable) = false];

      // ToUpsert lists out the spans we want to upsert and the configs
      // we want to upsert with.
      repeated SpanConfigEntry to_upsert = 2 [(gogoproto.nullable) = false];
    };

The RPCs are backed by a new host-tenant only
`system.span_configurations` table. Future PRs will wire a view of this
table into KV with an eye towards replacing our use of
`config.SystemConfig`.

    CREATE TABLE system.span_configurations (
        start_key    BYTES NOT NULL PRIMARY KEY,
        end_key      BYTES NOT NULL,
        config       BYTES NOT NULL,
        CONSTRAINT check_bounds CHECK (start_key < end_key),
        FAMILY "primary" (start_key, end_key, config)
    )

---

While here, we also introduce a `crdb_internal.pretty_span` builtin to
help with the readability of this table. In future PRs we'll make use of
this built-in for datadriven tests asserting on the state of the table.

Release note (sql change): We've added a `system.span_configurations`
table. This will later be used to store authoritative span configs that
KV has decided to apply.

Release justification: non-production code changes

Co-authored-by: irfan sharif <[email protected]>
  • Loading branch information
craig[bot] and irfansharif committed Aug 26, 2021
2 parents d26dec2 + 57ad801 commit ab1fc34
Show file tree
Hide file tree
Showing 77 changed files with 5,215 additions and 735 deletions.
1 change: 1 addition & 0 deletions BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ load("@bazel_gazelle//:def.bzl", "gazelle")
# gazelle:resolve proto go roachpb/internal_raft.proto //pkg/roachpb:with-mocks
# gazelle:resolve proto go roachpb/io-formats.proto //pkg/roachpb:with-mocks
# gazelle:resolve proto go roachpb/metadata.proto //pkg/roachpb:with-mocks
# gazelle:resolve proto go roachpb/span_config.proto //pkg/roachpb:with-mocks
# gazelle:exclude pkg/roachpb/batch_generated.go
# gazelle:exclude pkg/roachpb/batch_generated-gen.go

Expand Down
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -156,4 +156,4 @@ trace.datadog.project string CockroachDB the project under which traces will be
trace.debug.enable boolean false if set, traces for recent requests can be seen at https://<ui>/debug/requests
trace.lightstep.token string if set, traces go to Lightstep using this token
trace.zipkin.collector string if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time.
version version 21.1-152 set the active cluster version in the format '<major>.<minor>'
version version 21.1-154 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -160,6 +160,6 @@
<tr><td><code>trace.debug.enable</code></td><td>boolean</td><td><code>false</code></td><td>if set, traces for recent requests can be seen at https://<ui>/debug/requests</td></tr>
<tr><td><code>trace.lightstep.token</code></td><td>string</td><td><code></code></td><td>if set, traces go to Lightstep using this token</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.1-152</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.1-154</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -2858,6 +2858,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</p>
</span></td></tr>
<tr><td><a name="crdb_internal.pretty_key"></a><code>crdb_internal.pretty_key(raw_key: <a href="bytes.html">bytes</a>, skip_fields: <a href="int.html">int</a>) &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.pretty_span"></a><code>crdb_internal.pretty_span(raw_key_start: <a href="bytes.html">bytes</a>, raw_key_end: <a href="bytes.html">bytes</a>, skip_fields: <a href="int.html">int</a>) &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.range_stats"></a><code>crdb_internal.range_stats(key: <a href="bytes.html">bytes</a>) &rarr; jsonb</code></td><td><span class="funcdesc"><p>This function is used to retrieve range statistics information as a JSON object.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.reset_sql_stats"></a><code>crdb_internal.reset_sql_stats() &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>This function is used to clear the collected SQL statistics.</p>
Expand Down
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,7 @@ ALL_TESTS = [
"//pkg/server/tracedumper:tracedumper_test",
"//pkg/server:server_test",
"//pkg/settings:settings_test",
"//pkg/spanconfig/spanconfigkvaccessor:spanconfigkvaccessor_test",
"//pkg/spanconfig/spanconfigmanager:spanconfigmanager_test",
"//pkg/sql/catalog/catalogkeys:catalogkeys_test",
"//pkg/sql/catalog/catalogkv:catalogkv_test",
Expand Down
3 changes: 3 additions & 0 deletions pkg/ccl/backupccl/system_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,9 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{
systemschema.SQLInstancesTable.GetName(): {
shouldIncludeInClusterBackup: optOutOfClusterBackup,
},
systemschema.SpanConfigurationsTable.GetName(): {
shouldIncludeInClusterBackup: optOutOfClusterBackup,
},
}

// GetSystemTablesToIncludeInClusterBackup returns a set of system table names that
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/helpers_tenant_shim_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ func (t *testServerShim) RangeFeedFactory() interface{} { panic(unsuppor
func (t *testServerShim) Clock() *hlc.Clock { panic(unsupportedShimMethod) }
func (t *testServerShim) DistSenderI() interface{} { panic(unsupportedShimMethod) }
func (t *testServerShim) MigrationServer() interface{} { panic(unsupportedShimMethod) }
func (t *testServerShim) SpanConfigAccessor() interface{} { panic(unsupportedShimMethod) }
func (t *testServerShim) SQLServer() interface{} { panic(unsupportedShimMethod) }
func (t *testServerShim) SQLLivenessProvider() interface{} { panic(unsupportedShimMethod) }
func (t *testServerShim) StartupMigrationsManager() interface{} { panic(unsupportedShimMethod) }
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/kvccl/kvtenantccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_library(
"//pkg/roachpb:with-mocks",
"//pkg/rpc",
"//pkg/server/serverpb",
"//pkg/spanconfig",
"//pkg/util/contextutil",
"//pkg/util/grpcutil",
"//pkg/util/log",
Expand Down
72 changes: 63 additions & 9 deletions pkg/ccl/kvccl/kvtenantccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -108,6 +109,9 @@ var _ config.SystemConfigProvider = (*Connector)(nil)
// multi-region primitives.
var _ serverpb.RegionsServer = (*Connector)(nil)

// Connector is capable of accessing span configurations for secondary tenants.
var _ spanconfig.KVAccessor = (*Connector)(nil)

// NewConnector creates a new Connector.
// NOTE: Calling Start will set cfg.RPCContext.ClusterID.
func NewConnector(cfg kvtenant.ConnectorConfig, addrs []string) *Connector {
Expand Down Expand Up @@ -367,16 +371,16 @@ func (c *Connector) RangeLookup(
// Regions implements the serverpb.RegionsServer interface.
func (c *Connector) Regions(
ctx context.Context, req *serverpb.RegionsRequest,
) (*serverpb.RegionsResponse, error) {
ctx = c.AnnotateCtx(ctx)
for ctx.Err() == nil {
client, err := c.getClient(ctx)
if err != nil {
continue
}
return client.Regions(ctx, req)
) (resp *serverpb.RegionsResponse, _ error) {
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
var err error
resp, err = c.Regions(ctx, req)
return err
}); err != nil {
return nil, err
}
return nil, ctx.Err()

return resp, nil
}

// FirstRange implements the kvcoord.RangeDescriptorDB interface.
Expand Down Expand Up @@ -415,6 +419,56 @@ func (c *Connector) TokenBucket(
return nil, ctx.Err()
}

// GetSpanConfigEntriesFor implements the spanconfig.KVAccessor interface.
func (c *Connector) GetSpanConfigEntriesFor(
ctx context.Context, spans []roachpb.Span,
) (entries []roachpb.SpanConfigEntry, _ error) {
if err := c.withClient(ctx, func(ctx context.Context, c *client) error {
resp, err := c.GetSpanConfigs(ctx, &roachpb.GetSpanConfigsRequest{
Spans: spans,
})
if err != nil {
return err
}

entries = resp.SpanConfigEntries
return nil
}); err != nil {
return nil, err
}
return entries, nil
}

// UpdateSpanConfigEntries implements the spanconfig.KVAccessor
// interface.
func (c *Connector) UpdateSpanConfigEntries(
ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry,
) error {
return c.withClient(ctx, func(ctx context.Context, c *client) error {
_, err := c.UpdateSpanConfigs(ctx, &roachpb.UpdateSpanConfigsRequest{
ToDelete: toDelete,
ToUpsert: toUpsert,
})
return err
})
}

// withClient is a convenience wrapper that executes the given closure while
// papering over InternalClient retrieval errors.
func (c *Connector) withClient(
ctx context.Context, f func(ctx context.Context, c *client) error,
) error {
ctx = c.AnnotateCtx(ctx)
for ctx.Err() == nil {
c, err := c.getClient(ctx)
if err != nil {
continue
}
return f(ctx, c)
}
return ctx.Err()
}

// getClient returns the singleton InternalClient if one is currently active. If
// not, the method attempts to dial one of the configured addresses. The method
// blocks until either a connection is successfully established or the provided
Expand Down
12 changes: 12 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/connector_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,18 @@ func (*mockServer) TokenBucket(
panic("unimplemented")
}

func (m *mockServer) GetSpanConfigs(
context.Context, *roachpb.GetSpanConfigsRequest,
) (*roachpb.GetSpanConfigsResponse, error) {
panic("unimplemented")
}

func (m *mockServer) UpdateSpanConfigs(
context.Context, *roachpb.UpdateSpanConfigsRequest,
) (*roachpb.UpdateSpanConfigsResponse, error) {
panic("unimplemented")
}

func gossipEventForClusterID(clusterID uuid.UUID) *roachpb.GossipSubscriptionEvent {
return &roachpb.GossipSubscriptionEvent{
Key: gossip.KeyClusterID,
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/doctor/test_examine_cluster
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
debug doctor examine cluster
----
debug doctor examine cluster
Examining 40 descriptors and 41 namespace entries...
Examining 41 descriptors and 42 namespace entries...
ParentID 50, ParentSchemaID 29: relation "foo" (53): expected matching namespace entry, found none
Examining 4 jobs...
ERROR: validation failed
6 changes: 4 additions & 2 deletions pkg/cli/testdata/zip/partial1
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null
[node 1] 1 log file ...
[node 1] [log file ...
[node 1] requesting ranges... received response... done
[node 1] 42 ranges found
[node 1] 43 ranges found
[node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done
[node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done
[node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done
Expand Down Expand Up @@ -114,6 +114,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null
[node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done
[node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done
[node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done
[node 1] writing range 43... converting to JSON... writing binary output: debug/nodes/1/ranges/43.json... done
[node 2] node status... converting to JSON... writing binary output: debug/nodes/2/status.json... done
[node 2] using SQL connection URL: postgresql://...
[node 2] retrieving SQL data for crdb_internal.feature_usage... writing output: debug/nodes/2/crdb_internal.feature_usage.txt...
Expand Down Expand Up @@ -236,7 +237,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null
[node 3] 1 log file ...
[node 3] [log file ...
[node 3] requesting ranges... received response... done
[node 3] 42 ranges found
[node 3] 43 ranges found
[node 3] writing range 1... converting to JSON... writing binary output: debug/nodes/3/ranges/1.json... done
[node 3] writing range 2... converting to JSON... writing binary output: debug/nodes/3/ranges/2.json... done
[node 3] writing range 3... converting to JSON... writing binary output: debug/nodes/3/ranges/3.json... done
Expand Down Expand Up @@ -279,5 +280,6 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null
[node 3] writing range 40... converting to JSON... writing binary output: debug/nodes/3/ranges/40.json... done
[node 3] writing range 41... converting to JSON... writing binary output: debug/nodes/3/ranges/41.json... done
[node 3] writing range 42... converting to JSON... writing binary output: debug/nodes/3/ranges/42.json... done
[node 3] writing range 43... converting to JSON... writing binary output: debug/nodes/3/ranges/43.json... done
[cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done
[cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done
6 changes: 4 additions & 2 deletions pkg/cli/testdata/zip/partial1_excluded
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0
[node 1] 1 log file ...
[node 1] [log file ...
[node 1] requesting ranges... received response... done
[node 1] 42 ranges found
[node 1] 43 ranges found
[node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done
[node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done
[node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done
Expand Down Expand Up @@ -114,6 +114,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0
[node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done
[node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done
[node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done
[node 1] writing range 43... converting to JSON... writing binary output: debug/nodes/1/ranges/43.json... done
[node 2] skipping node... writing binary output: debug/nodes/2.skipped... done
[node 3] node status... converting to JSON... writing binary output: debug/nodes/3/status.json... done
[node 3] using SQL connection URL: postgresql://...
Expand Down Expand Up @@ -151,7 +152,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0
[node 3] 1 log file ...
[node 3] [log file ...
[node 3] requesting ranges... received response... done
[node 3] 42 ranges found
[node 3] 43 ranges found
[node 3] writing range 1... converting to JSON... writing binary output: debug/nodes/3/ranges/1.json... done
[node 3] writing range 2... converting to JSON... writing binary output: debug/nodes/3/ranges/2.json... done
[node 3] writing range 3... converting to JSON... writing binary output: debug/nodes/3/ranges/3.json... done
Expand Down Expand Up @@ -194,5 +195,6 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0
[node 3] writing range 40... converting to JSON... writing binary output: debug/nodes/3/ranges/40.json... done
[node 3] writing range 41... converting to JSON... writing binary output: debug/nodes/3/ranges/41.json... done
[node 3] writing range 42... converting to JSON... writing binary output: debug/nodes/3/ranges/42.json... done
[node 3] writing range 43... converting to JSON... writing binary output: debug/nodes/3/ranges/43.json... done
[cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done
[cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done
6 changes: 4 additions & 2 deletions pkg/cli/testdata/zip/partial2
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null
[node 1] 1 log file ...
[node 1] [log file ...
[node 1] requesting ranges... received response... done
[node 1] 42 ranges found
[node 1] 43 ranges found
[node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done
[node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done
[node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done
Expand Down Expand Up @@ -114,6 +114,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null
[node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done
[node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done
[node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done
[node 1] writing range 43... converting to JSON... writing binary output: debug/nodes/1/ranges/43.json... done
[node 3] node status... converting to JSON... writing binary output: debug/nodes/3/status.json... done
[node 3] using SQL connection URL: postgresql://...
[node 3] retrieving SQL data for crdb_internal.feature_usage... writing output: debug/nodes/3/crdb_internal.feature_usage.txt... done
Expand Down Expand Up @@ -150,7 +151,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null
[node 3] 1 log file ...
[node 3] [log file ...
[node 3] requesting ranges... received response... done
[node 3] 42 ranges found
[node 3] 43 ranges found
[node 3] writing range 1... converting to JSON... writing binary output: debug/nodes/3/ranges/1.json... done
[node 3] writing range 2... converting to JSON... writing binary output: debug/nodes/3/ranges/2.json... done
[node 3] writing range 3... converting to JSON... writing binary output: debug/nodes/3/ranges/3.json... done
Expand Down Expand Up @@ -193,5 +194,6 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null
[node 3] writing range 40... converting to JSON... writing binary output: debug/nodes/3/ranges/40.json... done
[node 3] writing range 41... converting to JSON... writing binary output: debug/nodes/3/ranges/41.json... done
[node 3] writing range 42... converting to JSON... writing binary output: debug/nodes/3/ranges/42.json... done
[node 3] writing range 43... converting to JSON... writing binary output: debug/nodes/3/ranges/43.json... done
[cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done
[cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done
3 changes: 2 additions & 1 deletion pkg/cli/testdata/zip/testzip
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null
[node 1] requesting log file ...
[node 1] 0 log file ...
[node 1] requesting ranges... received response... done
[node 1] 42 ranges found
[node 1] 43 ranges found
[node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done
[node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done
[node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done
Expand Down Expand Up @@ -114,5 +114,6 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null
[node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done
[node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done
[node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done
[node 1] writing range 43... converting to JSON... writing binary output: debug/nodes/1/ranges/43.json... done
[cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done
[cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done
Loading

0 comments on commit ab1fc34

Please sign in to comment.