diff --git a/.bazelrc b/.bazelrc index 09ac09d9bf67..f0e9c7b59c15 100644 --- a/.bazelrc +++ b/.bazelrc @@ -1,6 +1,6 @@ -build --symlink_prefix=_bazel/ --ui_event_filters=-DEBUG --define gotags=bazel,crdb_test_off,gss +build --symlink_prefix=_bazel/ --ui_event_filters=-DEBUG --define gotags=bazel,crdb_test_off,gss --experimental_proto_descriptor_sets_include_source_info test --config=test -build:test --symlink_prefix=_bazel/ --define gotags=bazel,crdb_test,gss --test_env=TZ= +build:test --define gotags=bazel,crdb_test,gss --test_env=TZ= query --ui_event_filters=-DEBUG try-import %workspace%/.bazelrc.user diff --git a/build/teamcity/cockroach/ci/builds/build_impl.sh b/build/teamcity/cockroach/ci/builds/build_impl.sh index a45a5ccc3691..4a799caadea9 100755 --- a/build/teamcity/cockroach/ci/builds/build_impl.sh +++ b/build/teamcity/cockroach/ci/builds/build_impl.sh @@ -14,7 +14,7 @@ CONFIG="$1" DOC_TARGETS= if [ "$CONFIG" == "crosslinux" ] then - DOC_TARGETS="//docs/generated:gen-logging-md //docs/generated:gen-logsinks-md //docs/generated:gen-eventlog-md //docs/generated:gen-logformats-md //docs/generated/settings:settings //docs/generated/settings:settings_for_tenants //docs/generated/sql //docs/generated/sql/bnf" + DOC_TARGETS=$(grep '^//' docs/generated/bazel_targets.txt) fi bazel build //pkg/cmd/bazci --config=ci diff --git a/docs/generated/bazel_targets.txt b/docs/generated/bazel_targets.txt new file mode 100644 index 000000000000..ccd53fafbd25 --- /dev/null +++ b/docs/generated/bazel_targets.txt @@ -0,0 +1,11 @@ +This file lists all the targets you need to build to build all the generated +documentation. Lines not beginning with // should be ignored. + +//docs/generated:gen-logging-md +//docs/generated:gen-logsinks-md +//docs/generated:gen-eventlog-md +//docs/generated:gen-logformats-md +//docs/generated/settings +//docs/generated/settings:settings_for_tenants +//docs/generated/sql +//docs/generated/sql/bnf diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 52d582e1403b..ccab0466e68e 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -7,6 +7,7 @@ ALL_TESTS = [ "//pkg/bench/rttanalysis:rttanalysis_test", "//pkg/bench:bench_test", "//pkg/blobs:blobs_test", + "//pkg/build/util:util_test", "//pkg/ccl/backupccl/backupresolver:backupresolver_test", "//pkg/ccl/backupccl:backupccl_test", "//pkg/ccl/baseccl:baseccl_test", diff --git a/pkg/build/util/BUILD.bazel b/pkg/build/util/BUILD.bazel new file mode 100644 index 000000000000..66fdff32211b --- /dev/null +++ b/pkg/build/util/BUILD.bazel @@ -0,0 +1,15 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "util", + srcs = ["util.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/build/util", + visibility = ["//visibility:public"], +) + +go_test( + name = "util_test", + srcs = ["util_test.go"], + embed = [":util"], + deps = ["@com_github_stretchr_testify//require"], +) diff --git a/pkg/build/util/util.go b/pkg/build/util/util.go new file mode 100644 index 000000000000..e63342cdbe36 --- /dev/null +++ b/pkg/build/util/util.go @@ -0,0 +1,75 @@ +// Copyright 2015 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +// +// This file contains assorted utilities for working with Bazel internals. + +package util + +import ( + "fmt" + "path/filepath" + "regexp" + "strings" +) + +// OutputOfBinaryRule returns the path of the binary produced by the +// given build target, relative to bazel-bin. That is, +// filepath.Join(bazelBin, OutputOfBinaryRule(target)) is the absolute +// path to the build binary for the target. +func OutputOfBinaryRule(target string) string { + colon := strings.Index(target, ":") + var bin string + if colon >= 0 { + bin = target[colon+1:] + } else { + bin = target[strings.LastIndex(target, "/")+1:] + } + var head string + if strings.HasPrefix(target, "@") { + doubleSlash := strings.Index(target, "//") + head = filepath.Join("external", target[1:doubleSlash]) + } else if colon >= 0 { + head = strings.TrimPrefix(target[:colon], "//") + } else { + head = strings.TrimPrefix(target, "//") + } + return filepath.Join(head, bin+"_", bin) +} + +// OutputsOfGenrule lists the outputs of a genrule. The first argument +// is the name of the target (e.g. //docs/generated/sql), and the second +// should be the output of `bazel query --output=xml $TARGET`. The +// returned slice is the list of outputs, all of which are relative +// paths atop `bazel-bin` as in `OutputOfBinaryRule`. +func OutputsOfGenrule(target, xmlQueryOutput string) ([]string, error) { + // XML parsing is a bit heavyweight here, and encoding/xml + // refuses to parse the query output since it's XML 1.1 instead + // of 1.0. Have fun with regexes instead. + colon := strings.LastIndex(target, ":") + if colon < 0 { + colon = len(target) + } + regexStr := fmt.Sprintf("^.*)\"/>$", regexp.QuoteMeta(target[:colon])) + re, err := regexp.Compile(regexStr) + if err != nil { + return nil, err + } + var ret []string + for _, line := range strings.Split(xmlQueryOutput, "\n") { + line = strings.TrimSpace(line) + submatch := re.FindStringSubmatch(line) + if submatch == nil { + continue + } + relBinPath := filepath.Join(strings.TrimPrefix(target[:colon], "//"), submatch[1]) + ret = append(ret, relBinPath) + } + return ret, nil +} diff --git a/pkg/build/util/util_test.go b/pkg/build/util/util_test.go new file mode 100644 index 000000000000..52cbfca20baa --- /dev/null +++ b/pkg/build/util/util_test.go @@ -0,0 +1,62 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package util + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestOutputOfBinaryRule(t *testing.T) { + require.Equal(t, OutputOfBinaryRule("//pkg/cmd/cockroach-short"), "pkg/cmd/cockroach-short/cockroach-short_/cockroach-short") + require.Equal(t, OutputOfBinaryRule("//pkg/cmd/cockroach-short:cockroach-short"), "pkg/cmd/cockroach-short/cockroach-short_/cockroach-short") + require.Equal(t, OutputOfBinaryRule("pkg/cmd/cockroach-short"), "pkg/cmd/cockroach-short/cockroach-short_/cockroach-short") + + require.Equal(t, OutputOfBinaryRule("@com_github_cockroachdb_stress//:stress"), "external/com_github_cockroachdb_stress/stress_/stress") +} + +func TestOutputsOfGenrule(t *testing.T) { + xmlQueryOutput := ` + + + + + + + + + + + + + + + + + + + +` + expected := []string{ + "docs/generated/sql/aggregates.md", + "docs/generated/sql/functions.md", + "docs/generated/sql/operators.md", + "docs/generated/sql/window_functions.md", + } + out, err := OutputsOfGenrule("//docs/generated/sql:sql", xmlQueryOutput) + require.NoError(t, err) + require.Equal(t, out, expected) + out, err = OutputsOfGenrule("//docs/generated/sql", xmlQueryOutput) + require.NoError(t, err) + require.Equal(t, out, expected) +} diff --git a/pkg/ccl/storageccl/export.go b/pkg/ccl/storageccl/export.go index 96b082a1c593..8a9ff72e5665 100644 --- a/pkg/ccl/storageccl/export.go +++ b/pkg/ccl/storageccl/export.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -177,11 +178,19 @@ func evalExport( // Time-bound iterators only make sense to use if the start time is set. useTBI := args.EnableTimeBoundIteratorOptimization && !args.StartTime.IsEmpty() + // Only use resume timestamp if splitting mid key is enabled. + resumeKeyTS := hlc.Timestamp{} + if args.SplitMidKey { + if !args.ReturnSST { + return result.Result{}, errors.New("SplitMidKey could only be used with ReturnSST option") + } + resumeKeyTS = args.ResumeKeyTS + } var curSizeOfExportedSSTs int64 for start := args.Key; start != nil; { destFile := &storage.MemFile{} - summary, resume, err := reader.ExportMVCCToSst(ctx, start, args.EndKey, args.StartTime, - h.Timestamp, exportAllRevisions, targetSize, maxSize, useTBI, destFile) + summary, resume, resumeTS, err := reader.ExportMVCCToSst(ctx, start, args.EndKey, args.StartTime, + h.Timestamp, resumeKeyTS, exportAllRevisions, targetSize, maxSize, args.SplitMidKey, useTBI, destFile) if err != nil { if errors.HasType(err, (*storage.ExceedMaxSizeError)(nil)) { err = errors.WithHintf(err, @@ -206,6 +215,7 @@ func evalExport( } exported := roachpb.ExportResponse_File{ Span: span, + EndKeyTS: resumeTS, Exported: summary, LocalityKV: localityKV, } @@ -249,6 +259,7 @@ func evalExport( } reply.Files = append(reply.Files, exported) start = resume + resumeKeyTS = resumeTS // If we are not returning the SSTs to the processor, there is no need to // paginate the ExportRequest since the reply size will not grow large diff --git a/pkg/ccl/storageccl/export_test.go b/pkg/ccl/storageccl/export_test.go index 87e400a7a612..194a9605458c 100644 --- a/pkg/ccl/storageccl/export_test.go +++ b/pkg/ccl/storageccl/export_test.go @@ -444,7 +444,7 @@ func exportUsingGoIterator( filter roachpb.MVCCFilter, startTime, endTime hlc.Timestamp, startKey, endKey roachpb.Key, - enableTimeBoundIteratorOptimization bool, + enableTimeBoundIteratorOptimization timeBoundOptimisation, reader storage.Reader, ) ([]byte, error) { memFile := &storage.MemFile{} @@ -466,7 +466,7 @@ func exportUsingGoIterator( iter := storage.NewMVCCIncrementalIterator(reader, storage.MVCCIncrementalIterOptions{ EndKey: endKey, - EnableTimeBoundIteratorOptimization: enableTimeBoundIteratorOptimization, + EnableTimeBoundIteratorOptimization: bool(enableTimeBoundIteratorOptimization), StartTime: startTime, EndTime: endTime, }) @@ -539,13 +539,29 @@ func loadSST(t *testing.T, data []byte, start, end roachpb.Key) []storage.MVCCKe return kvs } +type exportRevisions bool +type batchBoundaries bool +type timeBoundOptimisation bool + +const ( + exportAll exportRevisions = true + exportLatest exportRevisions = false + + stopAtTimestamps batchBoundaries = true + stopAtKeys batchBoundaries = false + + optimizeTimeBounds timeBoundOptimisation = true + dontOptimizeTimeBounds timeBoundOptimisation = false +) + func assertEqualKVs( ctx context.Context, e storage.Engine, startKey, endKey roachpb.Key, startTime, endTime hlc.Timestamp, - exportAllRevisions bool, - enableTimeBoundIteratorOptimization bool, + exportAllRevisions exportRevisions, + stopMidKey batchBoundaries, + enableTimeBoundIteratorOptimization timeBoundOptimisation, targetSize uint64, ) func(*testing.T) { return func(t *testing.T) { @@ -568,14 +584,16 @@ func assertEqualKVs( // Run the actual code path used when exporting MVCCs to SSTs. var kvs []storage.MVCCKeyValue + var resumeTs hlc.Timestamp for start := startKey; start != nil; { var sst []byte var summary roachpb.BulkOpSummary maxSize := uint64(0) prevStart := start + prevTs := resumeTs sstFile := &storage.MemFile{} - summary, start, err = e.ExportMVCCToSst(ctx, start, endKey, startTime, endTime, - exportAllRevisions, targetSize, maxSize, enableTimeBoundIteratorOptimization, sstFile) + summary, start, resumeTs, err = e.ExportMVCCToSst(ctx, start, endKey, startTime, endTime, resumeTs, + bool(exportAllRevisions), targetSize, maxSize, bool(stopMidKey), bool(enableTimeBoundIteratorOptimization), sstFile) require.NoError(t, err) sst = sstFile.Data() loaded := loadSST(t, sst, startKey, endKey) @@ -614,8 +632,8 @@ func assertEqualKVs( if dataSizeWhenExceeded == maxSize { maxSize-- } - _, _, err = e.ExportMVCCToSst(ctx, prevStart, endKey, startTime, endTime, - exportAllRevisions, targetSize, maxSize, enableTimeBoundIteratorOptimization, &storage.MemFile{}) + _, _, _, err = e.ExportMVCCToSst(ctx, prevStart, endKey, startTime, endTime, prevTs, + bool(exportAllRevisions), targetSize, maxSize, false, bool(enableTimeBoundIteratorOptimization), &storage.MemFile{}) require.Regexp(t, fmt.Sprintf("export size \\(%d bytes\\) exceeds max size \\(%d bytes\\)", dataSizeWhenExceeded, maxSize), err) } @@ -639,6 +657,7 @@ func assertEqualKVs( } } } + func TestRandomKeyAndTimestampExport(t *testing.T) { defer leaktest.AfterTest(t)() @@ -658,12 +677,13 @@ func TestRandomKeyAndTimestampExport(t *testing.T) { cleanupDir() } } + const keySize = 100 + const bytesPerValue = 300 getNumKeys := func(t *testing.T, rnd *rand.Rand, targetSize uint64) (numKeys int) { const ( - targetPages = 10 - bytesPerValue = 300 - minNumKeys = 2 // need > 1 keys for random key test - maxNumKeys = 5000 + targetPages = 10 + minNumKeys = 2 // need > 1 keys for random key test + maxNumKeys = 5000 ) numKeys = maxNumKeys if targetSize > 0 { @@ -695,11 +715,13 @@ func TestRandomKeyAndTimestampExport(t *testing.T) { timestamps = append(timestamps, ts) // Make keys unique and ensure they are monotonically increasing. - key := roachpb.Key(randutil.RandBytes(rnd, 100)) + key := roachpb.Key(randutil.RandBytes(rnd, keySize)) key = append([]byte(fmt.Sprintf("#%d", i)), key...) keys = append(keys, key) - value := roachpb.MakeValueFromBytes(randutil.RandBytes(rnd, 200)) + averageValueSize := bytesPerValue - keySize + valueSize := randutil.RandIntInRange(rnd, averageValueSize-100, averageValueSize+100) + value := roachpb.MakeValueFromBytes(randutil.RandBytes(rnd, valueSize)) value.InitChecksum(key) if err := storage.MVCCPut(ctx, batch, nil, key, ts, value, nil); err != nil { t.Fatal(err) @@ -741,28 +763,35 @@ func TestRandomKeyAndTimestampExport(t *testing.T) { tsMax = hlc.Timestamp{WallTime: math.MaxInt64, Logical: 0} ) - t.Run("ts (0-∞], latest, nontimebound", assertEqualKVs(ctx, e, keyMin, keyMax, tsMin, tsMax, false, false, targetSize)) - t.Run("ts (0-∞], all, nontimebound", assertEqualKVs(ctx, e, keyMin, keyMax, tsMin, tsMax, true, false, targetSize)) - t.Run("ts (0-∞], latest, timebound", assertEqualKVs(ctx, e, keyMin, keyMax, tsMin, tsMax, false, true, targetSize)) - t.Run("ts (0-∞], all, timebound", assertEqualKVs(ctx, e, keyMin, keyMax, tsMin, tsMax, true, true, targetSize)) - - upperBound := randutil.RandIntInRange(rnd, 1, numKeys) - lowerBound := rnd.Intn(upperBound) - - // Exercise random key ranges. - t.Run("kv [randLower, randUpper), latest, nontimebound", assertEqualKVs(ctx, e, keys[lowerBound], keys[upperBound], tsMin, tsMax, false, false, targetSize)) - t.Run("kv [randLower, randUpper), all, nontimebound", assertEqualKVs(ctx, e, keys[lowerBound], keys[upperBound], tsMin, tsMax, true, false, targetSize)) - t.Run("kv [randLower, randUpper), latest, timebound", assertEqualKVs(ctx, e, keys[lowerBound], keys[upperBound], tsMin, tsMax, false, true, targetSize)) - t.Run("kv [randLower, randUpper), all, timebound", assertEqualKVs(ctx, e, keys[lowerBound], keys[upperBound], tsMin, tsMax, true, true, targetSize)) - - upperBound = randutil.RandIntInRange(rnd, 1, numKeys) - lowerBound = rnd.Intn(upperBound) - - // Exercise random timestamps. - t.Run("kv (randLowerTime, randUpperTime], latest, nontimebound", assertEqualKVs(ctx, e, keyMin, keyMax, timestamps[lowerBound], timestamps[upperBound], false, false, targetSize)) - t.Run("kv (randLowerTime, randUpperTime], all, nontimebound", assertEqualKVs(ctx, e, keyMin, keyMax, timestamps[lowerBound], timestamps[upperBound], true, false, targetSize)) - t.Run("kv (randLowerTime, randUpperTime], latest, timebound", assertEqualKVs(ctx, e, keyMin, keyMax, timestamps[lowerBound], timestamps[upperBound], false, true, targetSize)) - t.Run("kv (randLowerTime, randUpperTime], all, timebound", assertEqualKVs(ctx, e, keyMin, keyMax, timestamps[lowerBound], timestamps[upperBound], true, true, targetSize)) + keyUpperBound := randutil.RandIntInRange(rnd, 1, numKeys) + keyLowerBound := rnd.Intn(keyUpperBound) + tsUpperBound := randutil.RandIntInRange(rnd, 1, numKeys) + tsLowerBound := rnd.Intn(tsUpperBound) + + for _, s := range []struct { + name string + keyMin roachpb.Key + keyMax roachpb.Key + tsMin hlc.Timestamp + tsMax hlc.Timestamp + }{ + {"ts (0-∞]", keyMin, keyMax, tsMin, tsMax}, + {"kv [randLower, randUpper)", keys[keyLowerBound], keys[keyUpperBound], tsMin, tsMax}, + {"kv (randLowerTime, randUpperTime]", keyMin, keyMax, timestamps[tsLowerBound], timestamps[tsUpperBound]}, + } { + t.Run(fmt.Sprintf("%s, latest, nontimebound", s.name), + assertEqualKVs(ctx, e, s.keyMin, s.keyMax, s.tsMin, s.tsMax, exportLatest, stopAtKeys, dontOptimizeTimeBounds, targetSize)) + t.Run(fmt.Sprintf("%s, all, nontimebound", s.name), + assertEqualKVs(ctx, e, s.keyMin, s.keyMax, s.tsMin, s.tsMax, exportAll, stopAtKeys, dontOptimizeTimeBounds, targetSize)) + t.Run(fmt.Sprintf("%s, all, split rows, nontimebound", s.name), + assertEqualKVs(ctx, e, s.keyMin, s.keyMax, s.tsMin, s.tsMax, exportAll, stopAtTimestamps, dontOptimizeTimeBounds, targetSize)) + t.Run(fmt.Sprintf("%s, latest, timebound", s.name), + assertEqualKVs(ctx, e, s.keyMin, s.keyMax, s.tsMin, s.tsMax, exportLatest, stopAtKeys, optimizeTimeBounds, targetSize)) + t.Run(fmt.Sprintf("%s, all, timebound", s.name), + assertEqualKVs(ctx, e, s.keyMin, s.keyMax, s.tsMin, s.tsMax, exportAll, stopAtKeys, optimizeTimeBounds, targetSize)) + t.Run(fmt.Sprintf("%s, all, split rows, timebound", s.name), + assertEqualKVs(ctx, e, s.keyMin, s.keyMax, s.tsMin, s.tsMax, exportAll, stopAtTimestamps, optimizeTimeBounds, targetSize)) + } } // Exercise min to max time and key ranges. for _, targetSize := range []uint64{ @@ -772,5 +801,4 @@ func TestRandomKeyAndTimestampExport(t *testing.T) { testWithTargetSize(t, targetSize) }) } - } diff --git a/pkg/cmd/bazci/BUILD.bazel b/pkg/cmd/bazci/BUILD.bazel index 37fdce79a07a..283ae685a03c 100644 --- a/pkg/cmd/bazci/BUILD.bazel +++ b/pkg/cmd/bazci/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/cmd/bazci", visibility = ["//visibility:private"], deps = [ + "//pkg/build/util", "@com_github_cockroachdb_errors//:errors", "@com_github_spf13_cobra//:cobra", ], diff --git a/pkg/cmd/bazci/watch.go b/pkg/cmd/bazci/watch.go index 22dadeb9e5f9..5a411543419d 100644 --- a/pkg/cmd/bazci/watch.go +++ b/pkg/cmd/bazci/watch.go @@ -19,10 +19,10 @@ import ( "os" "path" "path/filepath" - "regexp" "strings" "time" + bazelutil "github.com/cockroachdb/cockroach/pkg/build/util" "github.com/cockroachdb/errors" ) @@ -216,22 +216,11 @@ func (w watcher) stageBinaryArtifacts() error { if err != nil { return err } - // XML parsing is a bit heavyweight here, and encoding/xml - // refuses to parse the query output since it's XML 1.1 instead - // of 1.0. Have fun with regexes instead. - colon := strings.LastIndex(bin, ":") - regexStr := fmt.Sprintf("^.*)\"/>$", regexp.QuoteMeta(bin[:colon])) - re, err := regexp.Compile(regexStr) + outs, err := bazelutil.OutputsOfGenrule(bin, query) if err != nil { return err } - for _, line := range strings.Split(query, "\n") { - line = strings.TrimSpace(line) - submatch := re.FindStringSubmatch(line) - if submatch == nil { - continue - } - relBinPath := filepath.Join(strings.TrimPrefix(bin[:colon], "//"), submatch[1]) + for _, relBinPath := range outs { err := w.maybeStageArtifact(binSourceDir, relBinPath, 0666, finalizePhase, copyContentTo) if err != nil { return err diff --git a/pkg/cmd/dev/BUILD.bazel b/pkg/cmd/dev/BUILD.bazel index 877edfc6a5c1..dd90a2404fd6 100644 --- a/pkg/cmd/dev/BUILD.bazel +++ b/pkg/cmd/dev/BUILD.bazel @@ -16,6 +16,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/cmd/dev", visibility = ["//visibility:private"], deps = [ + "//pkg/build/util", "//pkg/cmd/dev/io/exec", "//pkg/cmd/dev/io/os", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/cmd/dev/build.go b/pkg/cmd/dev/build.go index df729c7809ba..fc366d4089ec 100644 --- a/pkg/cmd/dev/build.go +++ b/pkg/cmd/dev/build.go @@ -14,16 +14,21 @@ import ( "context" "fmt" "log" + "os" "path" "path/filepath" "strings" + bazelutil "github.com/cockroachdb/cockroach/pkg/build/util" "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/oserror" "github.com/spf13/cobra" ) -const crossFlag = "cross" +const ( + crossFlag = "cross" + hoistGeneratedCodeFlag = "hoist-generated-code" +) // makeBuildCmd constructs the subcommand used to build the specified binaries. func makeBuildCmd(runE func(cmd *cobra.Command, args []string) error) *cobra.Command { @@ -45,6 +50,7 @@ func makeBuildCmd(runE func(cmd *cobra.Command, args []string) error) *cobra.Com You can optionally set a config, as in --cross=windows. Defaults to linux if not specified. The config should be the name of a build configuration specified in .bazelrc, minus the "cross" prefix.`) + buildCmd.Flags().Bool(hoistGeneratedCodeFlag, false, "hoist generated code out of the Bazel sandbox into the workspace") buildCmd.Flags().Lookup(crossFlag).NoOptDefVal = "linux" return buildCmd } @@ -73,6 +79,7 @@ var buildTargetMapping = map[string]string{ func (d *dev) build(cmd *cobra.Command, targets []string) error { ctx := cmd.Context() cross := mustGetFlagString(cmd, crossFlag) + hoistGeneratedCode := mustGetFlagBool(cmd, hoistGeneratedCodeFlag) args, fullTargets, err := getBasicBuildArgs(targets) if err != nil { @@ -84,7 +91,7 @@ func (d *dev) build(cmd *cobra.Command, targets []string) error { if err := d.exec.CommandContextInheritingStdStreams(ctx, "bazel", args...); err != nil { return err } - return d.symlinkBinaries(ctx, fullTargets) + return d.stageArtifacts(ctx, fullTargets, hoistGeneratedCode) } // Cross-compilation case. cross = "cross" + cross @@ -103,7 +110,7 @@ func (d *dev) build(cmd *cobra.Command, targets []string) error { script.WriteString(fmt.Sprintf("bazel %s\n", strings.Join(args, " "))) script.WriteString(fmt.Sprintf("BAZELBIN=`bazel info bazel-bin --color=no --config=%s`\n", cross)) for _, target := range fullTargets { - script.WriteString(fmt.Sprintf("cp $BAZELBIN/%s /artifacts\n", targetToRelativeBinPath(target))) + script.WriteString(fmt.Sprintf("cp $BAZELBIN/%s /artifacts\n", bazelutil.OutputOfBinaryRule(target))) script.WriteString(fmt.Sprintf("chmod +w /artifacts/%s\n", targetToBinBasename(target))) } _, err = d.exec.CommandContextWithInput(ctx, script.String(), "docker", dockerArgs...) @@ -116,7 +123,7 @@ func (d *dev) build(cmd *cobra.Command, targets []string) error { return nil } -func (d *dev) symlinkBinaries(ctx context.Context, targets []string) error { +func (d *dev) stageArtifacts(ctx context.Context, targets []string, hoistGeneratedCode bool) error { workspace, err := d.getWorkspace(ctx) if err != nil { return err @@ -125,20 +132,21 @@ func (d *dev) symlinkBinaries(ctx context.Context, targets []string) error { if err = d.os.MkdirAll(path.Join(workspace, "bin")); err != nil { return err } + bazelBin, err := d.getBazelBin(ctx) + if err != nil { + return err + } for _, target := range targets { - binaryPath, err := d.getPathToBin(ctx, target) - if err != nil { - return err - } + binaryPath := filepath.Join(bazelBin, bazelutil.OutputOfBinaryRule(target)) base := targetToBinBasename(target) var symlinkPath string // Binaries beginning with the string "cockroach" go right at // the top of the workspace; others go in the `bin` directory. if strings.HasPrefix(base, "cockroach") { - symlinkPath = path.Join(workspace, base) + symlinkPath = filepath.Join(workspace, base) } else { - symlinkPath = path.Join(workspace, "bin", base) + symlinkPath = filepath.Join(workspace, "bin", base) } // Symlink from binaryPath -> symlinkPath @@ -155,29 +163,47 @@ func (d *dev) symlinkBinaries(ctx context.Context, targets []string) error { logSuccessfulBuild(target, rel) } - return nil -} - -// targetToRelativeBinPath returns the path of the binary produced by this build -// target relative to bazel-bin. That is, -// filepath.Join(bazelBin, targetToRelativeBinPath(target)) is the absolute -// path to the build binary for the target. -func targetToRelativeBinPath(target string) string { - var head string - if strings.HasPrefix(target, "@") { - doubleSlash := strings.Index(target, "//") - head = filepath.Join("external", target[1:doubleSlash]) - } else { - head = strings.TrimPrefix(target, "//") - } - var bin string - colon := strings.Index(target, ":") - if colon >= 0 { - bin = target[colon+1:] - } else { - bin = target[strings.LastIndex(target, "/")+1:] + if hoistGeneratedCode { + goFiles, err := d.os.ListFilesWithSuffix(filepath.Join(bazelBin, "pkg"), ".go") + if err != nil { + return err + } + for _, file := range goFiles { + const cockroachURL = "github.com/cockroachdb/cockroach/" + ind := strings.LastIndex(file, cockroachURL) + if ind > 0 { + // If the cockroach URL was found in the filepath, then we should + // trim everything up to and including the URL to find the path + // where the file should be staged. + loc := file[ind+len(cockroachURL):] + err := d.os.CopyFile(file, filepath.Join(workspace, loc)) + if err != nil { + return err + } + continue + } + pathComponents := strings.Split(file, string(os.PathSeparator)) + var skip bool + for _, component := range pathComponents[:len(pathComponents)-1] { + // Pretty decent heuristic for whether a file needs to be staged. + // When path components contain ., they normally are generated files + // from third-party packages, as in google.golang.org. Meanwhile, + // when path components end in _, that usually represents internal + // stuff that doesn't need to be staged, like + // pkg/cmd/dev/dev_test_/testmain.go. Note that generated proto code + // is handled by the cockroach URL case above. + if len(component) > 0 && (strings.ContainsRune(component, '.') || component[len(component)-1] == '_') { + skip = true + } + } + if !skip { + // Failures here don't mean much. Just ignore them. + _ = d.os.CopyFile(file, filepath.Join(workspace, strings.TrimPrefix(file, bazelBin+"/"))) + } + } } - return filepath.Join(head, bin+"_", bin) + + return nil } func targetToBinBasename(target string) string { @@ -191,18 +217,6 @@ func targetToBinBasename(target string) string { return base } -func (d *dev) getPathToBin(ctx context.Context, target string) (string, error) { - args := []string{"info", "bazel-bin", "--color=no"} - args = append(args, getConfigFlags()...) - out, err := d.exec.CommandContextSilent(ctx, "bazel", args...) - if err != nil { - return "", err - } - bazelBin := strings.TrimSpace(string(out)) - rel := targetToRelativeBinPath(target) - return filepath.Join(bazelBin, rel), nil -} - // getBasicBuildArgs is for enumerating the arguments to pass to `bazel` in // order to build the given high-level targets. // The first string slice returned is the list of arguments (i.e. to pass to diff --git a/pkg/cmd/dev/generate.go b/pkg/cmd/dev/generate.go index ab320f56be14..a1248fe93cc8 100644 --- a/pkg/cmd/dev/generate.go +++ b/pkg/cmd/dev/generate.go @@ -12,7 +12,9 @@ package main import ( "path/filepath" + "strings" + bazelutil "github.com/cockroachdb/cockroach/pkg/build/util" "github.com/cockroachdb/errors" "github.com/spf13/cobra" ) @@ -26,10 +28,10 @@ func makeGenerateCmd(runE func(cmd *cobra.Command, args []string) error) *cobra. Short: `Generate the specified files`, Long: `Generate the specified files.`, Example: ` - dev generate - dev generate bazel - dev generate protobuf - dev generate {exec,opt}gen`, + dev generate + dev generate bazel + dev generate docs +`, Args: cobra.MinimumNArgs(0), // TODO(irfansharif): Errors but default just eaten up. Let's wrap these // invocations in something that prints out the appropriate error log @@ -39,9 +41,12 @@ func makeGenerateCmd(runE func(cmd *cobra.Command, args []string) error) *cobra. } func (d *dev) generate(cmd *cobra.Command, targets []string) error { - // TODO(irfansharif): Flesh out the remaining targets. var generatorTargetMapping = map[string]func(cmd *cobra.Command) error{ - "bazel": d.generateBazel, + "bazel": d.generateBazel, + "docs": d.generateDocs, + "execgen": d.generateUnimplemented, + "optgen": d.generateUnimplemented, + "proto": d.generateUnimplemented, } if len(targets) == 0 { @@ -74,3 +79,59 @@ func (d *dev) generateBazel(cmd *cobra.Command) error { _, err = d.exec.CommandContext(ctx, filepath.Join(workspace, "build", "bazelutil", "bazel-generate.sh")) return err } + +func (d *dev) generateDocs(cmd *cobra.Command) error { + ctx := cmd.Context() + workspace, err := d.getWorkspace(ctx) + if err != nil { + return err + } + // List targets we need to build. + targetsFile, err := d.os.ReadFile(filepath.Join(workspace, "docs/generated/bazel_targets.txt")) + if err != nil { + return err + } + var targets []string + for _, line := range strings.Split(targetsFile, "\n") { + line = strings.TrimSpace(line) + if strings.HasPrefix(line, "//") { + targets = append(targets, line) + } + } + // Build targets. + var args []string + args = append(args, "build", "--color=yes", "--experimental_convenience_symlinks=ignore") + args = append(args, mustGetRemoteCacheArgs(remoteCacheAddr)...) + args = append(args, getConfigFlags()...) + args = append(args, targets...) + err = d.exec.CommandContextInheritingStdStreams(ctx, "bazel", args...) + if err != nil { + return err + } + // Copy docs from bazel-bin to workspace. + bazelBin, err := d.getBazelBin(ctx) + if err != nil { + return err + } + for _, target := range targets { + query, err := d.exec.CommandContextSilent(ctx, "bazel", "query", "--output=xml", target) + if err != nil { + return err + } + outputs, err := bazelutil.OutputsOfGenrule(target, string(query)) + if err != nil { + return err + } + for _, output := range outputs { + err = d.os.CopyFile(filepath.Join(bazelBin, output), filepath.Join(workspace, output)) + if err != nil { + return err + } + } + } + return nil +} + +func (*dev) generateUnimplemented(*cobra.Command) error { + return errors.New("To generate Go code, run `dev build` with the flag `--hoist-generated-code`") +} diff --git a/pkg/cmd/dev/io/os/os.go b/pkg/cmd/dev/io/os/os.go index 853fb93ae969..b8f46b24d37e 100644 --- a/pkg/cmd/dev/io/os/os.go +++ b/pkg/cmd/dev/io/os/os.go @@ -12,9 +12,13 @@ package os import ( "fmt" + "io" + "io/fs" "io/ioutil" "log" "os" + "path/filepath" + "strings" "github.com/cockroachdb/cockroach/pkg/cmd/dev/recording" "github.com/cockroachdb/errors/oserror" @@ -182,6 +186,63 @@ func (o *OS) ReadFile(filename string) (string, error) { return ret, err } +// CopyFile copies a file from one location to another. +func (o *OS) CopyFile(src, dst string) error { + command := fmt.Sprintf("cp %s %s", src, dst) + o.logger.Print(command) + + if o.Recording == nil { + // Do the real thing. + srcFile, err := os.Open(src) + if err != nil { + return err + } + dstFile, err := os.Create(dst) + if err != nil { + return err + } + _, err = io.Copy(dstFile, srcFile) + return err + } + + _, err := o.replay(command) + return err +} + +// ListFilesWithSuffix lists all the files under a directory recursively that +// end in the given suffix. +func (o *OS) ListFilesWithSuffix(root, suffix string) ([]string, error) { + command := fmt.Sprintf("find %s -name *%s", root, suffix) + o.logger.Print(command) + + var ret []string + if o.Recording == nil { + // Do the real thing. + err := filepath.Walk(root, func(path string, info fs.FileInfo, err error) error { + // If there's an error walking the tree, throw it away -- there's nothing + // interesting we can do with it. + if err != nil || info.IsDir() { + //nolint:returnerrcheck + return nil + } + if strings.HasSuffix(path, suffix) { + ret = append(ret, path) + } + return nil + }) + if err != nil { + return nil, err + } + return ret, nil + } + + lines, err := o.replay(command) + if err != nil { + return nil, err + } + return strings.Split(strings.TrimSpace(lines), "\n"), nil +} + // replay replays the specified command, erroring out if it's mismatched with // what the recording plays back next. It returns the recorded output. func (o *OS) replay(command string) (output string, err error) { diff --git a/pkg/cmd/dev/test.go b/pkg/cmd/dev/test.go index 9f6616e8d19d..e13506a754ca 100644 --- a/pkg/cmd/dev/test.go +++ b/pkg/cmd/dev/test.go @@ -12,8 +12,10 @@ package main import ( "fmt" + "path/filepath" "strings" + bazelutil "github.com/cockroachdb/cockroach/pkg/build/util" "github.com/cockroachdb/errors" "github.com/spf13/cobra" ) @@ -99,10 +101,11 @@ func (d *dev) runUnitTest(cmd *cobra.Command, pkgs []string) error { if err != nil { return err } - stressBin, err = d.getPathToBin(ctx, stressTarget) + bazelBin, err := d.getBazelBin(ctx) if err != nil { return err } + stressBin = filepath.Join(bazelBin, bazelutil.OutputOfBinaryRule(stressTarget)) } var args []string diff --git a/pkg/cmd/dev/testdata/build.txt b/pkg/cmd/dev/testdata/build.txt index 712e95f3c419..1e87b2ce761a 100644 --- a/pkg/cmd/dev/testdata/build.txt +++ b/pkg/cmd/dev/testdata/build.txt @@ -62,3 +62,18 @@ mkdir go/src/github.com/cockroachdb/cockroach/bin bazel info bazel-bin --color=no rm go/src/github.com/cockroachdb/cockroach/cockroach-short ln -s /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg/cmd/cockroach-short/cockroach-short_/cockroach-short go/src/github.com/cockroachdb/cockroach/cockroach-short + +dev build cockroach-short --hoist-generated-code +---- +getenv PATH +which cc +readlink /usr/local/opt/ccache/libexec/cc +export PATH=/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin +bazel build --color=yes --experimental_convenience_symlinks=ignore //pkg/cmd/cockroach-short --config=dev +bazel info workspace --color=no --config=dev +mkdir go/src/github.com/cockroachdb/cockroach/bin +bazel info bazel-bin --color=no --config=dev +rm go/src/github.com/cockroachdb/cockroach/cockroach-short +ln -s /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg/cmd/cockroach-short/cockroach-short_/cockroach-short go/src/github.com/cockroachdb/cockroach/cockroach-short +find /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg -name *.go +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg/kv/kvserver/kvserver_go_proto_/github.com/cockroachdb/cockroach/pkg/kv/kvserver/storage_services.pb.go go/src/github.com/cockroachdb/cockroach/pkg/kv/kvserver/storage_services.pb.go diff --git a/pkg/cmd/dev/testdata/generate.txt b/pkg/cmd/dev/testdata/generate.txt index a916f11850a1..992ffca29471 100644 --- a/pkg/cmd/dev/testdata/generate.txt +++ b/pkg/cmd/dev/testdata/generate.txt @@ -6,3 +6,21 @@ readlink /usr/local/opt/ccache/libexec/cc export PATH=/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin bazel info workspace --color=no --config=dev go/src/github.com/cockroachdb/cockroach/build/bazelutil/bazel-generate.sh + +dev gen docs +---- +getenv PATH +which cc +readlink /usr/local/opt/ccache/libexec/cc +export PATH=/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin +bazel info workspace --color=no --config=dev +cat go/src/github.com/cockroachdb/cockroach/docs/generated/bazel_targets.txt +bazel build --color=yes --experimental_convenience_symlinks=ignore --config=dev //docs/generated:gen-logging-md //docs/generated/sql +bazel info bazel-bin --color=no --config=dev +bazel query --output=xml //docs/generated:gen-logging-md +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/logging.md go/src/github.com/cockroachdb/cockroach/docs/generated/logging.md +bazel query --output=xml //docs/generated/sql +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/aggregates.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/aggregates.md +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/functions.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/functions.md +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/operators.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/operators.md +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/window_functions.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/window_functions.md diff --git a/pkg/cmd/dev/testdata/recording/build.txt b/pkg/cmd/dev/testdata/recording/build.txt index bbafb7c14e2f..e320645b97a5 100644 --- a/pkg/cmd/dev/testdata/recording/build.txt +++ b/pkg/cmd/dev/testdata/recording/build.txt @@ -172,3 +172,50 @@ rm go/src/github.com/cockroachdb/cockroach/cockroach-short ln -s /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg/cmd/cockroach-short/cockroach-short_/cockroach-short go/src/github.com/cockroachdb/cockroach/cockroach-short ---- + +getenv PATH +---- +/usr/local/opt/ccache/libexec:/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin + +which cc +---- +/usr/local/opt/ccache/libexec/cc + +readlink /usr/local/opt/ccache/libexec/cc +---- +../bin/ccache + +export PATH=/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin +---- + +bazel build --color=yes --experimental_convenience_symlinks=ignore //pkg/cmd/cockroach-short --config=dev +---- + +bazel info workspace --color=no --config=dev +---- +go/src/github.com/cockroachdb/cockroach + +mkdir go/src/github.com/cockroachdb/cockroach/bin +---- + +bazel info bazel-bin --color=no --config=dev +---- +/private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin + +rm go/src/github.com/cockroachdb/cockroach/cockroach-short +---- + +ln -s /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg/cmd/cockroach-short/cockroach-short_/cockroach-short go/src/github.com/cockroachdb/cockroach/cockroach-short +---- + +find /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg -name *.go +---- +---- +/private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/_bazel/bin/pkg/cmd/dev/dev_test_/testmain.go +/private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg/kv/kvclient/rangefeed/mock_rangefeed_gomock_gopath/src/google.golang.org/grpc/preloader.go +/private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg/kv/kvserver/kvserver_go_proto_/github.com/cockroachdb/cockroach/pkg/kv/kvserver/storage_services.pb.go +---- +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/pkg/kv/kvserver/kvserver_go_proto_/github.com/cockroachdb/cockroach/pkg/kv/kvserver/storage_services.pb.go go/src/github.com/cockroachdb/cockroach/pkg/kv/kvserver/storage_services.pb.go +---- diff --git a/pkg/cmd/dev/testdata/recording/generate.txt b/pkg/cmd/dev/testdata/recording/generate.txt index 892d3708bc3f..48d5904815ab 100644 --- a/pkg/cmd/dev/testdata/recording/generate.txt +++ b/pkg/cmd/dev/testdata/recording/generate.txt @@ -19,3 +19,108 @@ go/src/github.com/cockroachdb/cockroach go/src/github.com/cockroachdb/cockroach/build/bazelutil/bazel-generate.sh ---- + +getenv PATH +---- +/usr/local/opt/ccache/libexec:/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin + +which cc +---- +/usr/local/opt/ccache/libexec/cc + +readlink /usr/local/opt/ccache/libexec/cc +---- +../bin/ccache + +export PATH=/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin +---- + +bazel info workspace --color=no --config=dev +---- +go/src/github.com/cockroachdb/cockroach + +cat go/src/github.com/cockroachdb/cockroach/docs/generated/bazel_targets.txt +---- +---- +This line is ignored. + +//docs/generated:gen-logging-md +//docs/generated/sql +---- +---- + +bazel build --color=yes --experimental_convenience_symlinks=ignore --config=dev //docs/generated:gen-logging-md //docs/generated/sql +---- + +bazel info bazel-bin --color=no --config=dev +---- +/private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin + +bazel query --output=xml //docs/generated:gen-logging-md +---- +---- + + + + + + + + + + + + + + + + + + +---- +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/logging.md go/src/github.com/cockroachdb/cockroach/docs/generated/logging.md +---- + +bazel query --output=xml //docs/generated/sql +---- +---- + + + + + + + + + + + + + + + + + + + + + +---- +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/aggregates.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/aggregates.md +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/functions.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/functions.md +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/operators.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/operators.md +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/window_functions.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/window_functions.md +---- diff --git a/pkg/cmd/dev/util.go b/pkg/cmd/dev/util.go index 3e87745c1a48..f274167b67d8 100644 --- a/pkg/cmd/dev/util.go +++ b/pkg/cmd/dev/util.go @@ -82,14 +82,23 @@ func parseAddr(addr string) (string, error) { return fmt.Sprintf("%s:%s", ip, port), nil } -func (d *dev) getWorkspace(ctx context.Context) (string, error) { - args := []string{"info", "workspace", "--color=no"} +func (d *dev) getBazelInfo(ctx context.Context, key string) (string, error) { + args := []string{"info", key, "--color=no"} args = append(args, getConfigFlags()...) out, err := d.exec.CommandContextSilent(ctx, "bazel", args...) if err != nil { return "", err } return strings.TrimSpace(string(out)), nil + +} + +func (d *dev) getWorkspace(ctx context.Context) (string, error) { + return d.getBazelInfo(ctx, "workspace") +} + +func (d *dev) getBazelBin(ctx context.Context) (string, error) { + return d.getBazelInfo(ctx, "bazel-bin") } func getConfigFlags() []string { diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 6f1e1aadfde5..3185dcbb5749 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -401,14 +401,14 @@ func (s spanSetReader) Closed() bool { func (s spanSetReader) ExportMVCCToSst( ctx context.Context, startKey, endKey roachpb.Key, - startTS, endTS hlc.Timestamp, + startTS, endTS, firstKeyTS hlc.Timestamp, exportAllRevisions bool, targetSize, maxSize uint64, - useTBI bool, + stopMidKey, useTBI bool, dest io.Writer, -) (roachpb.BulkOpSummary, roachpb.Key, error) { - return s.r.ExportMVCCToSst(ctx, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize, - maxSize, useTBI, dest) +) (roachpb.BulkOpSummary, roachpb.Key, hlc.Timestamp, error) { + return s.r.ExportMVCCToSst(ctx, startKey, endKey, startTS, endTS, firstKeyTS, exportAllRevisions, targetSize, + maxSize, stopMidKey, useTBI, dest) } func (s spanSetReader) MVCCGet(key storage.MVCCKey) ([]byte, error) { diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 2de23858c21a..119cb7ed3730 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -3949,8 +3949,23 @@ var xxx_messageInfo_FileEncryptionOptions proto.InternalMessageInfo type ExportRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` Storage ExternalStorage `protobuf:"bytes,2,opt,name=storage,proto3" json:"storage"` - StartTime hlc.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time"` - MVCCFilter MVCCFilter `protobuf:"varint,4,opt,name=mvcc_filter,json=mvccFilter,proto3,enum=cockroach.roachpb.MVCCFilter" json:"mvcc_filter,omitempty"` + // ResumeKeyTS allows export request to resume at arbitrary key timestamp. This + // value doesn't affect export bounds, but ensures that no keys are skipped or + // duplicated when previous request doesn't complete fully and returned EndKeyTS + // together with resume span. + ResumeKeyTS hlc.Timestamp `protobuf:"bytes,12,opt,name=resume_key_ts,json=resumeKeyTs,proto3" json:"resume_key_ts"` + // MVCCFilter determines if request exports all data or latest data as of + // Timestamp specified in request header. If all data is requested StartTime + // is used as a lower bound and header Timestamp as higher bound or exported + // entries. + MVCCFilter MVCCFilter `protobuf:"varint,4,opt,name=mvcc_filter,json=mvccFilter,proto3,enum=cockroach.roachpb.MVCCFilter" json:"mvcc_filter,omitempty"` + // StartTime is only used whe MVCCFilter is set to All. + StartTime hlc.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time"` + // Split large rows in the middle of key sequence. This option will allow + // large history being broken up into target_file_size chunks and prevent + // blowing up on memory usage. This option is only allowed together with + // return_sst since caller should reconstruct full tables. + SplitMidKey bool `protobuf:"varint,13,opt,name=split_mid_key,json=splitMidKey,proto3" json:"split_mid_key,omitempty"` // Return the exported SST data in the response. ReturnSST bool `protobuf:"varint,5,opt,name=return_sst,json=returnSst,proto3" json:"return_sst,omitempty"` // EnableTimeBoundIteratorOptimization, if true, enables a performance @@ -4106,6 +4121,7 @@ var xxx_messageInfo_ExportResponse proto.InternalMessageInfo // path. type ExportResponse_File struct { Span Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` + EndKeyTS hlc.Timestamp `protobuf:"bytes,9,opt,name=end_key_ts,json=endKeyTs,proto3" json:"end_key_ts"` Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` Exported BulkOpSummary `protobuf:"bytes,6,opt,name=exported,proto3" json:"exported"` SST []byte `protobuf:"bytes,7,opt,name=sst,proto3" json:"sst,omitempty"` @@ -7514,553 +7530,557 @@ func init() { func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_e08772acc330f58b) } var fileDescriptor_e08772acc330f58b = []byte{ - // 8724 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5b, 0x6c, 0x23, 0x57, - 0x9a, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0xa7, 0x48, 0x96, 0x8e, 0xfa, 0x22, 0xab, 0x6d, 0xa9, 0xbb, - 0xda, 0x7d, 0xdd, 0x31, 0xe5, 0xee, 0x9e, 0xc9, 0x78, 0x6d, 0x8f, 0x67, 0x45, 0x8a, 0xdd, 0xa4, - 0x6e, 0xad, 0x2e, 0x52, 0xdd, 0xb0, 0x77, 0x36, 0xb5, 0xa5, 0xaa, 0x23, 0xa9, 0x46, 0x64, 0x15, - 0xbb, 0xaa, 0xa8, 0x96, 0x0c, 0xe4, 0x21, 0xc9, 0x62, 0xe2, 0xa7, 0x60, 0x02, 0x04, 0x58, 0x0f, - 0x36, 0x08, 0x66, 0x2f, 0x48, 0x1e, 0xf2, 0x90, 0x20, 0x09, 0x12, 0x6c, 0x6e, 0x0b, 0xe4, 0x25, - 0x83, 0x60, 0x90, 0x99, 0x7d, 0x5b, 0x04, 0x88, 0xb2, 0xd1, 0xe4, 0x21, 0x8b, 0x20, 0x48, 0x10, - 0x04, 0xd8, 0xc0, 0x40, 0x82, 0xe0, 0x5c, 0xea, 0x46, 0x16, 0x29, 0xaa, 0x5d, 0xce, 0x1a, 0xd8, - 0x17, 0x89, 0xf5, 0x9f, 0xf3, 0xff, 0x75, 0xce, 0x7f, 0x6e, 0xff, 0x77, 0xce, 0xff, 0x9f, 0x82, - 0x59, 0xdb, 0x52, 0xb5, 0x83, 0xee, 0xee, 0xb2, 0xda, 0x35, 0xca, 0x5d, 0xdb, 0x72, 0x2d, 0x34, - 0xab, 0x59, 0xda, 0x21, 0x25, 0x97, 0x79, 0xe2, 0xc2, 0x65, 0x6c, 0xdb, 0x96, 0xed, 0x74, 0x77, - 0x97, 0xd9, 0x0f, 0x96, 0x73, 0xe1, 0xfe, 0xe1, 0xd1, 0xf2, 0xe1, 0x91, 0x83, 0xed, 0x23, 0x6c, - 0x2f, 0x6b, 0x96, 0xa9, 0xf5, 0x6c, 0x1b, 0x9b, 0xda, 0xc9, 0x72, 0xdb, 0xd2, 0x0e, 0xe9, 0x1f, - 0xc3, 0xdc, 0x8f, 0xcb, 0x6b, 0x63, 0x55, 0x77, 0x7a, 0x9d, 0x8e, 0x6a, 0x9f, 0x2c, 0x53, 0xb1, + // 8799 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5b, 0x6c, 0x23, 0x59, + 0x7a, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0xa7, 0x48, 0x96, 0x8e, 0xfa, 0xa2, 0x51, 0xcf, 0x48, 0xdd, + 0xd5, 0xd3, 0x57, 0xef, 0x50, 0xd3, 0xdd, 0xbb, 0xd9, 0xf1, 0xcc, 0xec, 0xac, 0x25, 0x8a, 0xdd, + 0xa4, 0x6e, 0xad, 0x2e, 0x52, 0xdd, 0x98, 0xf1, 0x3a, 0xe5, 0x52, 0xd5, 0x91, 0x54, 0x2b, 0xb2, + 0x8a, 0x5d, 0x55, 0x54, 0x8b, 0x03, 0xe4, 0x21, 0x89, 0xb1, 0x99, 0xa7, 0x60, 0x03, 0x04, 0xf0, + 0x2c, 0x1c, 0x18, 0xeb, 0x0b, 0x92, 0x87, 0x3c, 0x24, 0x48, 0x82, 0x04, 0xce, 0xc5, 0x06, 0xf2, + 0x92, 0x41, 0x60, 0xc4, 0xeb, 0x37, 0x23, 0x40, 0x14, 0x47, 0x9b, 0x87, 0x18, 0x41, 0x80, 0x20, + 0x08, 0xe0, 0x60, 0x80, 0x04, 0xc1, 0xb9, 0xd4, 0x8d, 0x2c, 0x52, 0x54, 0x4f, 0x4d, 0x32, 0x80, + 0x5f, 0x24, 0xd6, 0x7f, 0xce, 0xff, 0xd7, 0x39, 0xff, 0xb9, 0xfd, 0xdf, 0x39, 0xff, 0x7f, 0x0a, + 0x66, 0x6d, 0x4b, 0xd5, 0x0e, 0x3b, 0x7b, 0xcb, 0x6a, 0xc7, 0x28, 0x77, 0x6c, 0xcb, 0xb5, 0xd0, + 0xac, 0x66, 0x69, 0x47, 0x94, 0x5c, 0xe6, 0x89, 0x0b, 0x97, 0xb1, 0x6d, 0x5b, 0xb6, 0xd3, 0xd9, + 0x5b, 0x66, 0x3f, 0x58, 0xce, 0x85, 0xfb, 0x47, 0xc7, 0xcb, 0x47, 0xc7, 0x0e, 0xb6, 0x8f, 0xb1, + 0xbd, 0xac, 0x59, 0xa6, 0xd6, 0xb5, 0x6d, 0x6c, 0x6a, 0xbd, 0xe5, 0x96, 0xa5, 0x1d, 0xd1, 0x3f, + 0x86, 0x79, 0x10, 0x97, 0xd7, 0xc6, 0xaa, 0xee, 0x74, 0xdb, 0x6d, 0xd5, 0xee, 0x2d, 0x53, 0xb1, 0xfc, 0x81, 0xe7, 0x45, 0x5e, 0xa1, 0x74, 0xd5, 0x55, 0x39, 0xed, 0x92, 0x47, 0x8b, 0x94, 0xe0, - 0x8a, 0x47, 0xed, 0x60, 0x57, 0x0d, 0xe5, 0xbe, 0xe6, 0xb8, 0x96, 0xad, 0xee, 0xe3, 0x65, 0x6c, - 0xee, 0x1b, 0x26, 0x26, 0x19, 0x8e, 0x34, 0x8d, 0x27, 0xbe, 0x19, 0x9b, 0xf8, 0x88, 0xa7, 0xce, - 0xf7, 0x5c, 0xa3, 0xbd, 0x7c, 0xd0, 0xd6, 0x96, 0x5d, 0xa3, 0x83, 0x1d, 0x57, 0xed, 0x74, 0xbd, - 0x2a, 0xd0, 0x14, 0xd7, 0x56, 0x35, 0xc3, 0xdc, 0xf7, 0xfe, 0x77, 0x77, 0x97, 0x6d, 0xac, 0x59, - 0xb6, 0x8e, 0x75, 0xc5, 0xe9, 0xaa, 0xa6, 0x57, 0xdc, 0x7d, 0x6b, 0xdf, 0xa2, 0x3f, 0x97, 0xc9, - 0x2f, 0x4e, 0x5d, 0xdc, 0xb7, 0xac, 0xfd, 0x36, 0x5e, 0xa6, 0x4f, 0xbb, 0xbd, 0xbd, 0x65, 0xbd, - 0x67, 0xab, 0xae, 0x61, 0x71, 0x2e, 0xe9, 0x9f, 0x08, 0x50, 0x90, 0xf1, 0xcb, 0x1e, 0x76, 0xdc, - 0x3a, 0x56, 0x75, 0x6c, 0xa3, 0x37, 0x20, 0x7d, 0x88, 0x4f, 0xe6, 0xd3, 0xd7, 0x85, 0xbb, 0x33, - 0x95, 0xe9, 0x2f, 0x4e, 0x97, 0xd2, 0xeb, 0xf8, 0x44, 0x26, 0x34, 0x74, 0x1d, 0xa6, 0xb1, 0xa9, - 0x2b, 0x24, 0x39, 0x13, 0x4d, 0x9e, 0xc2, 0xa6, 0xbe, 0x8e, 0x4f, 0xd0, 0xf7, 0x20, 0xeb, 0x10, - 0x69, 0xa6, 0x86, 0xe7, 0x27, 0xaf, 0x0b, 0x77, 0x27, 0x2b, 0xbf, 0xf2, 0xc5, 0xe9, 0xd2, 0x87, - 0xfb, 0x86, 0x7b, 0xd0, 0xdb, 0x2d, 0x6b, 0x56, 0x67, 0xd9, 0x6f, 0x6a, 0x7d, 0x37, 0xf8, 0xbd, - 0xdc, 0x3d, 0xdc, 0x5f, 0xee, 0xd7, 0x51, 0xb9, 0x75, 0x6c, 0x36, 0xf1, 0x4b, 0xd9, 0x97, 0xb8, - 0x96, 0xc9, 0x0a, 0x62, 0x6a, 0x2d, 0x93, 0x4d, 0x89, 0x69, 0xe9, 0x67, 0x29, 0x28, 0xca, 0xd8, - 0xe9, 0x5a, 0xa6, 0x83, 0x79, 0xc9, 0xdf, 0x85, 0xb4, 0x7b, 0x6c, 0xd2, 0x92, 0xe7, 0x1f, 0x2e, - 0x96, 0x07, 0x3a, 0x55, 0xb9, 0x65, 0xab, 0xa6, 0xa3, 0x6a, 0xa4, 0xfa, 0x32, 0xc9, 0x8a, 0xde, - 0x83, 0xbc, 0x8d, 0x9d, 0x5e, 0x07, 0x53, 0x45, 0xd2, 0x4a, 0xe5, 0x1f, 0x5e, 0x8d, 0xe1, 0x6c, - 0x76, 0x55, 0x53, 0x06, 0x96, 0x97, 0xfc, 0x46, 0x4d, 0x28, 0x70, 0x4e, 0x1b, 0xab, 0x8e, 0x65, - 0xce, 0x4f, 0x5f, 0x17, 0xee, 0x16, 0x1f, 0x96, 0x63, 0x78, 0xa3, 0xa5, 0x24, 0x8f, 0xbd, 0x0e, - 0x96, 0x29, 0x97, 0x3c, 0x63, 0x87, 0x9e, 0xd0, 0x1b, 0x90, 0x35, 0x7b, 0x1d, 0xa2, 0x5f, 0x87, - 0x6a, 0x2f, 0x2d, 0x4f, 0x9b, 0xbd, 0xce, 0x3a, 0x3e, 0x71, 0xd0, 0x35, 0xc8, 0x91, 0xa4, 0xdd, - 0x13, 0x17, 0x3b, 0xf3, 0x59, 0x9a, 0x46, 0xf2, 0x56, 0xc8, 0xb3, 0xf4, 0x11, 0xcc, 0x84, 0xa5, - 0x22, 0x04, 0x45, 0xb9, 0xd6, 0xdc, 0xd9, 0xac, 0x29, 0x3b, 0x5b, 0xeb, 0x5b, 0x4f, 0x5f, 0x6c, - 0x89, 0x13, 0xe8, 0x12, 0x88, 0x9c, 0xb6, 0x5e, 0xfb, 0x58, 0xd9, 0x68, 0x6c, 0x36, 0x5a, 0xa2, - 0xb0, 0x90, 0xf9, 0xec, 0x77, 0x17, 0x27, 0xd6, 0x32, 0xd9, 0x29, 0x71, 0x5a, 0xfa, 0x5d, 0x01, - 0xe0, 0x09, 0x76, 0x79, 0x6f, 0x40, 0x15, 0x98, 0x3a, 0xa0, 0x25, 0x9e, 0x17, 0xa8, 0x5a, 0xae, - 0xc7, 0x56, 0x2d, 0xd4, 0x73, 0x2a, 0xd9, 0x9f, 0x9c, 0x2e, 0x4d, 0xfc, 0xfc, 0x74, 0x49, 0x90, - 0x39, 0x27, 0x7a, 0x06, 0xf9, 0x43, 0x7c, 0xa2, 0xf0, 0x71, 0x39, 0x9f, 0xa2, 0x3a, 0x7a, 0x37, - 0x24, 0xe8, 0xf0, 0xa8, 0xec, 0x0d, 0xd1, 0x72, 0x68, 0x38, 0x97, 0x09, 0x47, 0xb9, 0xe9, 0xda, - 0xd8, 0xdc, 0x77, 0x0f, 0x64, 0x38, 0xc4, 0x27, 0x1b, 0x4c, 0x86, 0xf4, 0x07, 0x02, 0xe4, 0x69, - 0x29, 0x99, 0x52, 0x51, 0xb5, 0xaf, 0x98, 0x37, 0xce, 0x6d, 0x81, 0x98, 0x72, 0x96, 0x61, 0xf2, - 0x48, 0x6d, 0xf7, 0x30, 0x2d, 0x61, 0xfe, 0xe1, 0x7c, 0x8c, 0x8c, 0xe7, 0x24, 0x5d, 0x66, 0xd9, - 0xd0, 0x07, 0x30, 0x63, 0x98, 0x2e, 0x36, 0x5d, 0x85, 0xb1, 0xa5, 0xcf, 0x61, 0xcb, 0xb3, 0xdc, - 0xf4, 0x41, 0xfa, 0xc7, 0x02, 0xc0, 0x76, 0x2f, 0x51, 0x3d, 0x7f, 0x73, 0xcc, 0xf2, 0x57, 0x32, - 0x84, 0xd5, 0xab, 0xc5, 0x15, 0x98, 0x32, 0xcc, 0xb6, 0x61, 0xb2, 0xf2, 0x67, 0x65, 0xfe, 0x84, - 0x2e, 0xc1, 0xe4, 0x6e, 0xdb, 0x30, 0x75, 0x3a, 0x1e, 0xb2, 0x32, 0x7b, 0x90, 0x64, 0xc8, 0xd3, - 0x52, 0x27, 0xa8, 0x77, 0xe9, 0x34, 0x05, 0x97, 0xab, 0x96, 0xa9, 0x1b, 0x64, 0x48, 0xaa, 0xed, - 0xaf, 0x85, 0x56, 0xd6, 0xe0, 0x92, 0x8e, 0xbb, 0x36, 0xd6, 0x54, 0x17, 0xeb, 0x0a, 0x3e, 0xee, - 0x8e, 0xd9, 0xc6, 0x28, 0xe0, 0xaa, 0x1d, 0x77, 0x29, 0x8d, 0x8c, 0x5a, 0x22, 0x80, 0x8d, 0xda, - 0x29, 0x32, 0x65, 0xca, 0x59, 0x7c, 0xdc, 0xa5, 0xa3, 0x36, 0x5e, 0xcd, 0xe8, 0x9b, 0x70, 0x55, - 0x6d, 0xb7, 0xad, 0x57, 0x8a, 0xb1, 0xa7, 0xe8, 0x16, 0x76, 0x14, 0xd3, 0x72, 0x15, 0x7c, 0x6c, - 0x38, 0x2e, 0x9d, 0x12, 0xb2, 0xf2, 0x1c, 0x4d, 0x6e, 0xec, 0xad, 0x5a, 0xd8, 0xd9, 0xb2, 0xdc, - 0x1a, 0x49, 0x0a, 0x35, 0xe5, 0x74, 0xb8, 0x29, 0xa5, 0x5f, 0x83, 0x2b, 0xfd, 0xfa, 0x4d, 0xb2, - 0xfd, 0x7e, 0x2a, 0x40, 0xb1, 0x61, 0x1a, 0xee, 0xd7, 0xa2, 0xe1, 0x7c, 0x7d, 0xa6, 0xc3, 0xfa, - 0xbc, 0x0f, 0xe2, 0x9e, 0x6a, 0xb4, 0x9f, 0x9a, 0x2d, 0xab, 0xb3, 0xeb, 0xb8, 0x96, 0x89, 0x1d, - 0xae, 0xf0, 0x01, 0xba, 0xf4, 0x1c, 0x4a, 0x7e, 0x6d, 0x92, 0x54, 0x93, 0x0b, 0x62, 0xc3, 0xd4, - 0x6c, 0xdc, 0xc1, 0x66, 0xa2, 0x7a, 0x7a, 0x13, 0x72, 0x86, 0x27, 0x97, 0xea, 0x2a, 0x2d, 0x07, - 0x04, 0xa9, 0x07, 0xb3, 0xa1, 0xb7, 0x26, 0x39, 0x5d, 0x92, 0xc5, 0x08, 0xbf, 0x52, 0x82, 0x36, - 0x22, 0x8b, 0x11, 0x7e, 0xc5, 0xa6, 0xb7, 0x26, 0x14, 0x56, 0x71, 0x1b, 0xbb, 0x38, 0xc1, 0x9a, - 0x4a, 0x3b, 0x50, 0xf4, 0x84, 0x26, 0xd9, 0x30, 0xbf, 0x29, 0x00, 0xe2, 0x72, 0x55, 0x73, 0x3f, - 0xc9, 0x12, 0xa3, 0x25, 0x62, 0x5a, 0xb8, 0x3d, 0xdb, 0x64, 0xcb, 0x39, 0xeb, 0x93, 0xc0, 0x48, + 0x8a, 0x47, 0x6d, 0x63, 0x57, 0x0d, 0xe5, 0xbe, 0xe6, 0xb8, 0x96, 0xad, 0x1e, 0xe0, 0x65, 0x6c, + 0x1e, 0x18, 0x26, 0x26, 0x19, 0x8e, 0x35, 0x8d, 0x27, 0xbe, 0x19, 0x9b, 0xf8, 0x88, 0xa7, 0xce, + 0x77, 0x5d, 0xa3, 0xb5, 0x7c, 0xd8, 0xd2, 0x96, 0x5d, 0xa3, 0x8d, 0x1d, 0x57, 0x6d, 0x77, 0xbc, + 0x2a, 0xd0, 0x14, 0xd7, 0x56, 0x35, 0xc3, 0x3c, 0xf0, 0xfe, 0x77, 0xf6, 0x96, 0x6d, 0xac, 0x59, + 0xb6, 0x8e, 0x75, 0xc5, 0xe9, 0xa8, 0xa6, 0x57, 0xdc, 0x03, 0xeb, 0xc0, 0xa2, 0x3f, 0x97, 0xc9, + 0x2f, 0x4e, 0x5d, 0x3c, 0xb0, 0xac, 0x83, 0x16, 0x5e, 0xa6, 0x4f, 0x7b, 0xdd, 0xfd, 0x65, 0xbd, + 0x6b, 0xab, 0xae, 0x61, 0x71, 0x2e, 0xe9, 0x9f, 0x0a, 0x50, 0x90, 0xf1, 0xcb, 0x2e, 0x76, 0xdc, + 0x1a, 0x56, 0x75, 0x6c, 0xa3, 0x37, 0x20, 0x7d, 0x84, 0x7b, 0xf3, 0xe9, 0xeb, 0xc2, 0xdd, 0x99, + 0xd5, 0xe9, 0x2f, 0x4f, 0x97, 0xd2, 0x1b, 0xb8, 0x27, 0x13, 0x1a, 0xba, 0x0e, 0xd3, 0xd8, 0xd4, + 0x15, 0x92, 0x9c, 0x89, 0x26, 0x4f, 0x61, 0x53, 0xdf, 0xc0, 0x3d, 0xf4, 0x03, 0xc8, 0x3a, 0x44, + 0x9a, 0xa9, 0xe1, 0xf9, 0xc9, 0xeb, 0xc2, 0xdd, 0xc9, 0xd5, 0x5f, 0xfa, 0xf2, 0x74, 0xe9, 0xc3, + 0x03, 0xc3, 0x3d, 0xec, 0xee, 0x95, 0x35, 0xab, 0xbd, 0xec, 0x37, 0xb5, 0xbe, 0x17, 0xfc, 0x5e, + 0xee, 0x1c, 0x1d, 0x2c, 0xf7, 0xeb, 0xa8, 0xdc, 0x3c, 0x31, 0x1b, 0xf8, 0xa5, 0xec, 0x4b, 0x5c, + 0xcf, 0x64, 0x05, 0x31, 0xb5, 0x9e, 0xc9, 0xa6, 0xc4, 0xb4, 0xf4, 0x47, 0x29, 0x28, 0xca, 0xd8, + 0xe9, 0x58, 0xa6, 0x83, 0x79, 0xc9, 0xdf, 0x85, 0xb4, 0x7b, 0x62, 0xd2, 0x92, 0xe7, 0x1f, 0x2e, + 0x96, 0x07, 0x3a, 0x55, 0xb9, 0x69, 0xab, 0xa6, 0xa3, 0x6a, 0xa4, 0xfa, 0x32, 0xc9, 0x8a, 0xde, + 0x83, 0xbc, 0x8d, 0x9d, 0x6e, 0x1b, 0x53, 0x45, 0xd2, 0x4a, 0xe5, 0x1f, 0x5e, 0x8d, 0xe1, 0x6c, + 0x74, 0x54, 0x53, 0x06, 0x96, 0x97, 0xfc, 0x46, 0x0d, 0x28, 0x70, 0x4e, 0x1b, 0xab, 0x8e, 0x65, + 0xce, 0x4f, 0x5f, 0x17, 0xee, 0x16, 0x1f, 0x96, 0x63, 0x78, 0xa3, 0xa5, 0x24, 0x8f, 0xdd, 0x36, + 0x96, 0x29, 0x97, 0x3c, 0x63, 0x87, 0x9e, 0xd0, 0x1b, 0x90, 0x35, 0xbb, 0x6d, 0xa2, 0x5f, 0x87, + 0x6a, 0x2f, 0x2d, 0x4f, 0x9b, 0xdd, 0xf6, 0x06, 0xee, 0x39, 0xe8, 0x1a, 0xe4, 0x48, 0xd2, 0x5e, + 0xcf, 0xc5, 0xce, 0x7c, 0x96, 0xa6, 0x91, 0xbc, 0xab, 0xe4, 0x59, 0xfa, 0x08, 0x66, 0xc2, 0x52, + 0x11, 0x82, 0xa2, 0x5c, 0x6d, 0xec, 0x6e, 0x55, 0x95, 0xdd, 0xed, 0x8d, 0xed, 0xa7, 0x2f, 0xb6, + 0xc5, 0x09, 0x74, 0x09, 0x44, 0x4e, 0xdb, 0xa8, 0x7e, 0xac, 0x6c, 0xd6, 0xb7, 0xea, 0x4d, 0x51, + 0x58, 0xc8, 0x7c, 0xf6, 0x3b, 0x8b, 0x13, 0xeb, 0x99, 0xec, 0x94, 0x38, 0x2d, 0xfd, 0x8e, 0x00, + 0xf0, 0x04, 0xbb, 0xbc, 0x37, 0xa0, 0x55, 0x98, 0x3a, 0xa4, 0x25, 0x9e, 0x17, 0xa8, 0x5a, 0xae, + 0xc7, 0x56, 0x2d, 0xd4, 0x73, 0x56, 0xb3, 0x5f, 0x9c, 0x2e, 0x4d, 0xfc, 0xec, 0x74, 0x49, 0x90, + 0x39, 0x27, 0x7a, 0x06, 0xf9, 0x23, 0xdc, 0x53, 0xf8, 0xb8, 0x9c, 0x4f, 0x51, 0x1d, 0xbd, 0x1b, + 0x12, 0x74, 0x74, 0x5c, 0xf6, 0x86, 0x68, 0x39, 0x34, 0x9c, 0xcb, 0x84, 0xa3, 0xdc, 0x70, 0x6d, + 0x6c, 0x1e, 0xb8, 0x87, 0x32, 0x1c, 0xe1, 0xde, 0x26, 0x93, 0x21, 0xfd, 0x81, 0x00, 0x79, 0x5a, + 0x4a, 0xa6, 0x54, 0x54, 0xe9, 0x2b, 0xe6, 0x8d, 0x73, 0x5b, 0x20, 0xa6, 0x9c, 0x65, 0x98, 0x3c, + 0x56, 0x5b, 0x5d, 0x4c, 0x4b, 0x98, 0x7f, 0x38, 0x1f, 0x23, 0xe3, 0x39, 0x49, 0x97, 0x59, 0x36, + 0xf4, 0x01, 0xcc, 0x18, 0xa6, 0x8b, 0x4d, 0x57, 0x61, 0x6c, 0xe9, 0x73, 0xd8, 0xf2, 0x2c, 0x37, + 0x7d, 0x90, 0xfe, 0x89, 0x00, 0xb0, 0xd3, 0x4d, 0x54, 0xcf, 0xdf, 0x1e, 0xb3, 0xfc, 0xab, 0x19, + 0xc2, 0xea, 0xd5, 0xe2, 0x0a, 0x4c, 0x19, 0x66, 0xcb, 0x30, 0x59, 0xf9, 0xb3, 0x32, 0x7f, 0x42, + 0x97, 0x60, 0x72, 0xaf, 0x65, 0x98, 0x3a, 0x1d, 0x0f, 0x59, 0x99, 0x3d, 0x48, 0x32, 0xe4, 0x69, + 0xa9, 0x13, 0xd4, 0xbb, 0x74, 0x9a, 0x82, 0xcb, 0x15, 0xcb, 0xd4, 0x0d, 0x32, 0x24, 0xd5, 0xd6, + 0x37, 0x42, 0x2b, 0xeb, 0x70, 0x49, 0xc7, 0x1d, 0x1b, 0x6b, 0xaa, 0x8b, 0x75, 0x05, 0x9f, 0x74, + 0xc6, 0x6c, 0x63, 0x14, 0x70, 0x55, 0x4f, 0x3a, 0x94, 0x46, 0x46, 0x2d, 0x11, 0xc0, 0x46, 0xed, + 0x14, 0x99, 0x32, 0xe5, 0x2c, 0x3e, 0xe9, 0xd0, 0x51, 0x1b, 0xaf, 0x66, 0xf4, 0x6d, 0xb8, 0xaa, + 0xb6, 0x5a, 0xd6, 0x2b, 0xc5, 0xd8, 0x57, 0x74, 0x0b, 0x3b, 0x8a, 0x69, 0xb9, 0x0a, 0x3e, 0x31, + 0x1c, 0x97, 0x4e, 0x09, 0x59, 0x79, 0x8e, 0x26, 0xd7, 0xf7, 0xd7, 0x2c, 0xec, 0x6c, 0x5b, 0x6e, + 0x95, 0x24, 0x85, 0x9a, 0x72, 0x3a, 0xdc, 0x94, 0xd2, 0xaf, 0xc0, 0x95, 0x7e, 0xfd, 0x26, 0xd9, + 0x7e, 0x7f, 0x28, 0x40, 0xb1, 0x6e, 0x1a, 0xee, 0x37, 0xa2, 0xe1, 0x7c, 0x7d, 0xa6, 0xc3, 0xfa, + 0xbc, 0x0f, 0xe2, 0xbe, 0x6a, 0xb4, 0x9e, 0x9a, 0x4d, 0xab, 0xbd, 0xe7, 0xb8, 0x96, 0x89, 0x1d, + 0xae, 0xf0, 0x01, 0xba, 0xf4, 0x1c, 0x4a, 0x7e, 0x6d, 0x92, 0x54, 0x93, 0x0b, 0x62, 0xdd, 0xd4, + 0x6c, 0xdc, 0xc6, 0x66, 0xa2, 0x7a, 0x7a, 0x13, 0x72, 0x86, 0x27, 0x97, 0xea, 0x2a, 0x2d, 0x07, + 0x04, 0xa9, 0x0b, 0xb3, 0xa1, 0xb7, 0x26, 0x39, 0x5d, 0x92, 0xc5, 0x08, 0xbf, 0x52, 0x82, 0x36, + 0x22, 0x8b, 0x11, 0x7e, 0xc5, 0xa6, 0xb7, 0x06, 0x14, 0xd6, 0x70, 0x0b, 0xbb, 0x38, 0xc1, 0x9a, + 0x4a, 0xbb, 0x50, 0xf4, 0x84, 0x26, 0xd9, 0x30, 0xbf, 0x2e, 0x00, 0xe2, 0x72, 0x55, 0xf3, 0x20, + 0xc9, 0x12, 0xa3, 0x25, 0x62, 0x5a, 0xb8, 0x5d, 0xdb, 0x64, 0xcb, 0x39, 0xeb, 0x93, 0xc0, 0x48, 0x74, 0x45, 0x0f, 0x86, 0x6c, 0x26, 0x3c, 0x64, 0xb9, 0x79, 0xf3, 0x0a, 0xe6, 0x22, 0x05, 0x4b, 0xb6, 0xf9, 0x32, 0xb4, 0x4c, 0xa9, 0xeb, 0xe9, 0xb0, 0x0d, 0x47, 0x89, 0xd2, 0xe7, 0x02, 0xcc, - 0x56, 0xdb, 0x58, 0xb5, 0x13, 0xd7, 0xc8, 0x77, 0x21, 0xab, 0x63, 0x55, 0xa7, 0x55, 0x66, 0x03, - 0xfb, 0xad, 0x90, 0x14, 0x62, 0xe9, 0x96, 0x0f, 0xda, 0x5a, 0xb9, 0xe5, 0xd9, 0xc0, 0x7c, 0x74, - 0xfb, 0x4c, 0xd2, 0xc7, 0x80, 0xc2, 0x25, 0x4b, 0xb2, 0x23, 0xfc, 0x5e, 0x0a, 0x90, 0x8c, 0x8f, - 0xb0, 0xed, 0x26, 0x5e, 0xed, 0x55, 0xc8, 0xbb, 0xaa, 0xbd, 0x8f, 0x5d, 0x85, 0x58, 0xf7, 0x17, - 0xa9, 0x39, 0x30, 0x3e, 0x42, 0x46, 0x2d, 0xb8, 0x83, 0x4d, 0x75, 0xb7, 0x8d, 0xa9, 0x14, 0x65, - 0xd7, 0xea, 0x99, 0xba, 0x62, 0xb8, 0xd8, 0x56, 0x5d, 0xcb, 0x56, 0xac, 0xae, 0x6b, 0x74, 0x8c, - 0x4f, 0xa9, 0x61, 0xcf, 0xbb, 0xda, 0x4d, 0x96, 0x9d, 0x30, 0x57, 0x48, 0xe6, 0x06, 0xcf, 0xfb, - 0x34, 0x94, 0x15, 0x95, 0x61, 0xce, 0xd8, 0x37, 0x2d, 0x1b, 0x2b, 0xfb, 0x9a, 0xe2, 0x1e, 0xd8, - 0xd8, 0x39, 0xb0, 0xda, 0xde, 0x82, 0x34, 0xcb, 0x92, 0x9e, 0x68, 0x2d, 0x2f, 0x41, 0xfa, 0x04, - 0xe6, 0x22, 0x5a, 0x4a, 0xb2, 0x09, 0xfe, 0x87, 0x00, 0xf9, 0xa6, 0xa6, 0x9a, 0x49, 0xea, 0xfe, - 0x23, 0xc8, 0x3b, 0x9a, 0x6a, 0x2a, 0x7b, 0x96, 0xdd, 0x51, 0x5d, 0x5a, 0xaf, 0x62, 0x44, 0xf7, - 0xbe, 0x7d, 0xaf, 0xa9, 0xe6, 0x63, 0x9a, 0x49, 0x06, 0xc7, 0xff, 0xdd, 0x6f, 0xbf, 0x4e, 0x7e, - 0x79, 0xfb, 0x95, 0x0d, 0xef, 0xb5, 0x4c, 0x36, 0x2d, 0x66, 0xa4, 0x3f, 0x15, 0x60, 0x86, 0x55, - 0x39, 0xc9, 0xe1, 0xfd, 0x2d, 0xc8, 0xd8, 0xd6, 0x2b, 0x36, 0xbc, 0xf3, 0x0f, 0xaf, 0xc5, 0x88, - 0x58, 0xc7, 0x27, 0xe1, 0xf5, 0x93, 0x66, 0x47, 0x15, 0xe0, 0x56, 0xaa, 0x42, 0xb9, 0xd3, 0xe3, - 0x72, 0x03, 0xe3, 0x92, 0x89, 0x8c, 0x3b, 0x50, 0xda, 0x55, 0x5d, 0xed, 0x40, 0xb1, 0x79, 0x21, - 0xc9, 0x5a, 0x9b, 0xbe, 0x3b, 0x23, 0x17, 0x29, 0xd9, 0x2b, 0xba, 0x43, 0x6a, 0xce, 0xc6, 0x9b, - 0x83, 0xff, 0x9c, 0xb5, 0xf9, 0xff, 0x15, 0xf8, 0x18, 0xf2, 0x6a, 0xfe, 0xe7, 0xad, 0xe9, 0x7f, - 0x94, 0x82, 0xab, 0xd5, 0x03, 0xac, 0x1d, 0x56, 0x2d, 0xd3, 0x31, 0x1c, 0x97, 0xe8, 0x2e, 0xc9, - 0xf6, 0xbf, 0x06, 0xb9, 0x57, 0x86, 0x7b, 0xa0, 0xe8, 0xc6, 0xde, 0x1e, 0x9d, 0x6d, 0xb3, 0x72, - 0x96, 0x10, 0x56, 0x8d, 0xbd, 0x3d, 0xf4, 0x08, 0x32, 0x1d, 0x4b, 0x67, 0xc6, 0x7c, 0xf1, 0xe1, - 0x52, 0x8c, 0x78, 0x5a, 0x34, 0xa7, 0xd7, 0xd9, 0xb4, 0x74, 0x2c, 0xd3, 0xcc, 0x68, 0x11, 0x40, - 0x23, 0xd4, 0xae, 0x65, 0x98, 0x2e, 0x9f, 0x1c, 0x43, 0x14, 0x54, 0x87, 0x9c, 0x8b, 0xed, 0x8e, - 0x61, 0xaa, 0x2e, 0x9e, 0x9f, 0xa4, 0xca, 0x7b, 0x3b, 0xb6, 0xe0, 0xdd, 0xb6, 0xa1, 0xa9, 0xab, - 0xd8, 0xd1, 0x6c, 0xa3, 0xeb, 0x5a, 0x36, 0xd7, 0x62, 0xc0, 0x2c, 0xfd, 0xf5, 0x0c, 0xcc, 0x0f, - 0xea, 0x26, 0xc9, 0x1e, 0xb2, 0x0d, 0x53, 0x36, 0x76, 0x7a, 0x6d, 0x97, 0xf7, 0x91, 0x87, 0xc3, - 0x54, 0x10, 0x53, 0x02, 0xba, 0x75, 0xd1, 0x76, 0x79, 0xb1, 0xb9, 0x9c, 0x85, 0x7f, 0x21, 0xc0, - 0x14, 0x4b, 0x40, 0x0f, 0x20, 0x6b, 0x93, 0x85, 0x41, 0x31, 0x74, 0x5a, 0xc6, 0x74, 0xe5, 0xca, - 0xd9, 0xe9, 0xd2, 0x34, 0x5d, 0x2c, 0x1a, 0xab, 0x5f, 0x04, 0x3f, 0xe5, 0x69, 0x9a, 0xaf, 0xa1, - 0x93, 0xd6, 0x72, 0x5c, 0xd5, 0x76, 0xe9, 0xa6, 0x52, 0x8a, 0x21, 0x24, 0x4a, 0x58, 0xc7, 0x27, - 0x68, 0x0d, 0xa6, 0x1c, 0x57, 0x75, 0x7b, 0x0e, 0x6f, 0xaf, 0x0b, 0x15, 0xb6, 0x49, 0x39, 0x65, - 0x2e, 0x81, 0x98, 0x5b, 0x3a, 0x76, 0x55, 0xa3, 0x4d, 0x1b, 0x30, 0x27, 0xf3, 0x27, 0xe9, 0xb7, - 0x04, 0x98, 0x62, 0x59, 0xd1, 0x55, 0x98, 0x93, 0x57, 0xb6, 0x9e, 0xd4, 0x94, 0xc6, 0xd6, 0x6a, - 0xad, 0x55, 0x93, 0x37, 0x1b, 0x5b, 0x2b, 0xad, 0x9a, 0x38, 0x81, 0xae, 0x00, 0xf2, 0x12, 0xaa, - 0x4f, 0xb7, 0x9a, 0x8d, 0x66, 0xab, 0xb6, 0xd5, 0x12, 0x05, 0xba, 0xa7, 0x42, 0xe9, 0x21, 0x6a, - 0x0a, 0xbd, 0x0d, 0xd7, 0xfb, 0xa9, 0x4a, 0xb3, 0xb5, 0xd2, 0x6a, 0x2a, 0xb5, 0x66, 0xab, 0xb1, - 0xb9, 0xd2, 0xaa, 0xad, 0x8a, 0xe9, 0x11, 0xb9, 0xc8, 0x4b, 0x64, 0xb9, 0x56, 0x6d, 0x89, 0x19, - 0xc9, 0x85, 0xcb, 0x32, 0xd6, 0xac, 0x4e, 0xb7, 0xe7, 0x62, 0x52, 0x4a, 0x27, 0xc9, 0x91, 0x72, - 0x15, 0xa6, 0x75, 0xfb, 0x44, 0xb1, 0x7b, 0x26, 0x1f, 0x27, 0x53, 0xba, 0x7d, 0x22, 0xf7, 0x4c, - 0xe9, 0x1f, 0x08, 0x70, 0xa5, 0xff, 0xb5, 0x49, 0x76, 0xc2, 0x67, 0x90, 0x57, 0x75, 0x1d, 0xeb, - 0x8a, 0x8e, 0xdb, 0xae, 0xca, 0x4d, 0xa2, 0xfb, 0x21, 0x49, 0x7c, 0x2b, 0xb0, 0xec, 0x6f, 0x05, - 0x6e, 0x3e, 0xaf, 0x56, 0x69, 0x41, 0x56, 0x09, 0x87, 0x37, 0xfd, 0x50, 0x21, 0x94, 0x22, 0xfd, - 0x28, 0x03, 0x85, 0x9a, 0xa9, 0xb7, 0x8e, 0x13, 0x5d, 0x4b, 0xae, 0xc0, 0x94, 0x66, 0x75, 0x3a, - 0x86, 0xeb, 0x29, 0x88, 0x3d, 0xa1, 0x5f, 0x0e, 0x99, 0xb2, 0xe9, 0x31, 0x0c, 0xba, 0xc0, 0x88, - 0x45, 0xbf, 0x0e, 0x57, 0xc9, 0xac, 0x69, 0x9b, 0x6a, 0x5b, 0x61, 0xd2, 0x14, 0xd7, 0x36, 0xf6, - 0xf7, 0xb1, 0xcd, 0xb7, 0x1f, 0xef, 0xc6, 0x94, 0xb3, 0xc1, 0x39, 0xaa, 0x94, 0xa1, 0xc5, 0xf2, - 0xcb, 0x97, 0x8d, 0x38, 0x32, 0xfa, 0x10, 0x80, 0x2c, 0x45, 0x74, 0x4b, 0xd3, 0xe1, 0xf3, 0xd1, - 0xb0, 0x3d, 0x4d, 0x6f, 0x0a, 0x22, 0x0c, 0xe4, 0xd9, 0x41, 0xcf, 0x40, 0x34, 0x4c, 0x65, 0xaf, - 0x6d, 0xec, 0x1f, 0xb8, 0xca, 0x2b, 0xdb, 0x70, 0xb1, 0x33, 0x3f, 0x4b, 0x65, 0xc4, 0x35, 0x75, - 0x93, 0x6f, 0xcd, 0xea, 0x2f, 0x48, 0x4e, 0x2e, 0xad, 0x68, 0x98, 0x8f, 0x29, 0x3f, 0x25, 0x3a, - 0x68, 0x99, 0x40, 0xa1, 0x97, 0x3d, 0xc3, 0xc6, 0xca, 0x83, 0xae, 0x46, 0xf7, 0x41, 0xb2, 0x95, - 0xe2, 0xd9, 0xe9, 0x12, 0xc8, 0x8c, 0xfc, 0x60, 0xbb, 0x4a, 0xa0, 0x11, 0xfb, 0xdd, 0xd5, 0x88, - 0xda, 0xbb, 0x96, 0xe1, 0x58, 0xe6, 0x7c, 0x8e, 0xa9, 0x9d, 0x3d, 0xa1, 0x7b, 0x20, 0xba, 0xc7, - 0xa6, 0x72, 0x80, 0x55, 0xdb, 0xdd, 0xc5, 0xaa, 0x4b, 0xd6, 0x67, 0xa0, 0x39, 0x4a, 0xee, 0xb1, - 0x59, 0x0f, 0x91, 0xd7, 0x32, 0xd9, 0x69, 0x31, 0xbb, 0x96, 0xc9, 0x66, 0xc5, 0x9c, 0xf4, 0x1f, - 0x04, 0x28, 0x7a, 0x7d, 0x23, 0xc9, 0x6e, 0x7c, 0x17, 0x44, 0xcb, 0xc4, 0x4a, 0xf7, 0x40, 0x75, - 0x30, 0x6f, 0x4b, 0xbe, 0x3a, 0x14, 0x2d, 0x13, 0x6f, 0x13, 0x32, 0x6b, 0x19, 0xb4, 0x0d, 0xb3, - 0x8e, 0xab, 0xee, 0x1b, 0xe6, 0xbe, 0xe2, 0x6f, 0xf1, 0x53, 0xcb, 0x62, 0x4c, 0x24, 0x20, 0x72, - 0x6e, 0x9f, 0x1e, 0x31, 0x29, 0xfe, 0x50, 0x80, 0xd9, 0x15, 0xbd, 0x63, 0x98, 0xcd, 0x6e, 0xdb, + 0x56, 0x5a, 0x58, 0xb5, 0x13, 0xd7, 0xc8, 0xf7, 0x21, 0xab, 0x63, 0x55, 0xa7, 0x55, 0x66, 0x03, + 0xfb, 0xad, 0x90, 0x14, 0x62, 0xe9, 0x96, 0x0f, 0x5b, 0x5a, 0xb9, 0xe9, 0xd9, 0xc0, 0x7c, 0x74, + 0xfb, 0x4c, 0xd2, 0xc7, 0x80, 0xc2, 0x25, 0x4b, 0xb2, 0x23, 0xfc, 0x6e, 0x0a, 0x90, 0x8c, 0x8f, + 0xb1, 0xed, 0x26, 0x5e, 0xed, 0x35, 0xc8, 0xbb, 0xaa, 0x7d, 0x80, 0x5d, 0x85, 0x58, 0xf7, 0x17, + 0xa9, 0x39, 0x30, 0x3e, 0x42, 0x46, 0x4d, 0xb8, 0x83, 0x4d, 0x75, 0xaf, 0x85, 0xa9, 0x14, 0x65, + 0xcf, 0xea, 0x9a, 0xba, 0x62, 0xb8, 0xd8, 0x56, 0x5d, 0xcb, 0x56, 0xac, 0x8e, 0x6b, 0xb4, 0x8d, + 0x4f, 0xa9, 0x61, 0xcf, 0xbb, 0xda, 0x4d, 0x96, 0x9d, 0x30, 0xaf, 0x92, 0xcc, 0x75, 0x9e, 0xf7, + 0x69, 0x28, 0x2b, 0x2a, 0xc3, 0x9c, 0x71, 0x60, 0x5a, 0x36, 0x56, 0x0e, 0x34, 0xc5, 0x3d, 0xb4, + 0xb1, 0x73, 0x68, 0xb5, 0xbc, 0x05, 0x69, 0x96, 0x25, 0x3d, 0xd1, 0x9a, 0x5e, 0x82, 0xf4, 0x09, + 0xcc, 0x45, 0xb4, 0x94, 0x64, 0x13, 0xfc, 0x77, 0x01, 0xf2, 0x0d, 0x4d, 0x35, 0x93, 0xd4, 0xfd, + 0x47, 0x90, 0x77, 0x34, 0xd5, 0x54, 0xf6, 0x2d, 0xbb, 0xad, 0xba, 0xb4, 0x5e, 0xc5, 0x88, 0xee, + 0x7d, 0xfb, 0x5e, 0x53, 0xcd, 0xc7, 0x34, 0x93, 0x0c, 0x8e, 0xff, 0xbb, 0xdf, 0x7e, 0x9d, 0xfc, + 0xea, 0xf6, 0x2b, 0x1b, 0xde, 0xeb, 0x99, 0x6c, 0x5a, 0xcc, 0x48, 0x7f, 0x2e, 0xc0, 0x0c, 0xab, + 0x72, 0x92, 0xc3, 0xfb, 0x3b, 0x90, 0xb1, 0xad, 0x57, 0x6c, 0x78, 0xe7, 0x1f, 0x5e, 0x8b, 0x11, + 0xb1, 0x81, 0x7b, 0xe1, 0xf5, 0x93, 0x66, 0x47, 0xab, 0xc0, 0xad, 0x54, 0x85, 0x72, 0xa7, 0xc7, + 0xe5, 0x06, 0xc6, 0x25, 0x13, 0x19, 0x77, 0xa0, 0xb4, 0xa7, 0xba, 0xda, 0xa1, 0x62, 0xf3, 0x42, + 0x92, 0xb5, 0x36, 0x7d, 0x77, 0x46, 0x2e, 0x52, 0xb2, 0x57, 0x74, 0x87, 0xd4, 0x9c, 0x8d, 0x37, + 0x07, 0xff, 0x05, 0x6b, 0xf3, 0xff, 0x23, 0xf0, 0x31, 0xe4, 0xd5, 0xfc, 0x2f, 0x5a, 0xd3, 0xff, + 0x24, 0x05, 0x57, 0x2b, 0x87, 0x58, 0x3b, 0xaa, 0x58, 0xa6, 0x63, 0x38, 0x2e, 0xd1, 0x5d, 0x92, + 0xed, 0x7f, 0x0d, 0x72, 0xaf, 0x0c, 0xf7, 0x50, 0xd1, 0x8d, 0xfd, 0x7d, 0x3a, 0xdb, 0x66, 0xe5, + 0x2c, 0x21, 0xac, 0x19, 0xfb, 0xfb, 0xe8, 0x11, 0x64, 0xda, 0x96, 0xce, 0x8c, 0xf9, 0xe2, 0xc3, + 0xa5, 0x18, 0xf1, 0xb4, 0x68, 0x4e, 0xb7, 0xbd, 0x65, 0xe9, 0x58, 0xa6, 0x99, 0xd1, 0x22, 0x80, + 0x46, 0xa8, 0x1d, 0xcb, 0x30, 0x5d, 0x3e, 0x39, 0x86, 0x28, 0xa8, 0x06, 0x39, 0x17, 0xdb, 0x6d, + 0xc3, 0x54, 0x5d, 0x3c, 0x3f, 0x49, 0x95, 0xf7, 0x76, 0x6c, 0xc1, 0x3b, 0x2d, 0x43, 0x53, 0xd7, + 0xb0, 0xa3, 0xd9, 0x46, 0xc7, 0xb5, 0x6c, 0xae, 0xc5, 0x80, 0x59, 0xfa, 0x9b, 0x19, 0x98, 0x1f, + 0xd4, 0x4d, 0x92, 0x3d, 0x64, 0x07, 0xa6, 0x6c, 0xec, 0x74, 0x5b, 0x2e, 0xef, 0x23, 0x0f, 0x87, + 0xa9, 0x20, 0xa6, 0x04, 0x74, 0xeb, 0xa2, 0xe5, 0xf2, 0x62, 0x73, 0x39, 0x0b, 0xff, 0x52, 0x80, + 0x29, 0x96, 0x80, 0x1e, 0x40, 0xd6, 0x26, 0x0b, 0x83, 0x62, 0xe8, 0xb4, 0x8c, 0xe9, 0xd5, 0x2b, + 0x67, 0xa7, 0x4b, 0xd3, 0x74, 0xb1, 0xa8, 0xaf, 0x7d, 0x19, 0xfc, 0x94, 0xa7, 0x69, 0xbe, 0xba, + 0x4e, 0x5a, 0xcb, 0x71, 0x55, 0xdb, 0xa5, 0x9b, 0x4a, 0x29, 0x86, 0x90, 0x28, 0x61, 0x03, 0xf7, + 0xd0, 0x3a, 0x4c, 0x39, 0xae, 0xea, 0x76, 0x1d, 0xde, 0x5e, 0x17, 0x2a, 0x6c, 0x83, 0x72, 0xca, + 0x5c, 0x02, 0x31, 0xb7, 0x74, 0xec, 0xaa, 0x46, 0x8b, 0x36, 0x60, 0x4e, 0xe6, 0x4f, 0xd2, 0x6f, + 0x08, 0x30, 0xc5, 0xb2, 0xa2, 0xab, 0x30, 0x27, 0xaf, 0x6c, 0x3f, 0xa9, 0x2a, 0xf5, 0xed, 0xb5, + 0x6a, 0xb3, 0x2a, 0x6f, 0xd5, 0xb7, 0x57, 0x9a, 0x55, 0x71, 0x02, 0x5d, 0x01, 0xe4, 0x25, 0x54, + 0x9e, 0x6e, 0x37, 0xea, 0x8d, 0x66, 0x75, 0xbb, 0x29, 0x0a, 0x74, 0x4f, 0x85, 0xd2, 0x43, 0xd4, + 0x14, 0x7a, 0x1b, 0xae, 0xf7, 0x53, 0x95, 0x46, 0x73, 0xa5, 0xd9, 0x50, 0xaa, 0x8d, 0x66, 0x7d, + 0x6b, 0xa5, 0x59, 0x5d, 0x13, 0xd3, 0x23, 0x72, 0x91, 0x97, 0xc8, 0x72, 0xb5, 0xd2, 0x14, 0x33, + 0x92, 0x0b, 0x97, 0x65, 0xac, 0x59, 0xed, 0x4e, 0xd7, 0xc5, 0xa4, 0x94, 0x4e, 0x92, 0x23, 0xe5, + 0x2a, 0x4c, 0xeb, 0x76, 0x4f, 0xb1, 0xbb, 0x26, 0x1f, 0x27, 0x53, 0xba, 0xdd, 0x93, 0xbb, 0xa6, + 0xf4, 0x0f, 0x05, 0xb8, 0xd2, 0xff, 0xda, 0x24, 0x3b, 0xe1, 0x33, 0xc8, 0xab, 0xba, 0x8e, 0x75, + 0x45, 0xc7, 0x2d, 0x57, 0xe5, 0x26, 0xd1, 0xfd, 0x90, 0x24, 0xbe, 0x15, 0x58, 0xf6, 0xb7, 0x02, + 0xb7, 0x9e, 0x57, 0x2a, 0xb4, 0x20, 0x6b, 0x84, 0xc3, 0x9b, 0x7e, 0xa8, 0x10, 0x4a, 0x91, 0x7e, + 0x92, 0x81, 0x42, 0xd5, 0xd4, 0x9b, 0x27, 0x89, 0xae, 0x25, 0x57, 0x60, 0x4a, 0xb3, 0xda, 0x6d, + 0xc3, 0xf5, 0x14, 0xc4, 0x9e, 0xd0, 0x2f, 0x86, 0x4c, 0xd9, 0xf4, 0x18, 0x06, 0x5d, 0x60, 0xc4, + 0xa2, 0x5f, 0x85, 0xab, 0x64, 0xd6, 0xb4, 0x4d, 0xb5, 0xa5, 0x30, 0x69, 0x8a, 0x6b, 0x1b, 0x07, + 0x07, 0xd8, 0xe6, 0xdb, 0x8f, 0x77, 0x63, 0xca, 0x59, 0xe7, 0x1c, 0x15, 0xca, 0xd0, 0x64, 0xf9, + 0xe5, 0xcb, 0x46, 0x1c, 0x19, 0x7d, 0x08, 0x40, 0x96, 0x22, 0xba, 0xa5, 0xe9, 0xf0, 0xf9, 0x68, + 0xd8, 0x9e, 0xa6, 0x37, 0x05, 0x11, 0x06, 0xf2, 0xec, 0xa0, 0x67, 0x20, 0x1a, 0xa6, 0xb2, 0xdf, + 0x32, 0x0e, 0x0e, 0x5d, 0xe5, 0x95, 0x6d, 0xb8, 0xd8, 0x99, 0x9f, 0xa5, 0x32, 0xe2, 0x9a, 0xba, + 0xc1, 0xb7, 0x66, 0xf5, 0x17, 0x24, 0x27, 0x97, 0x56, 0x34, 0xcc, 0xc7, 0x94, 0x9f, 0x12, 0x1d, + 0xb4, 0x4c, 0xa0, 0xd0, 0xcb, 0xae, 0x61, 0x63, 0xe5, 0x41, 0x47, 0xa3, 0xfb, 0x20, 0xd9, 0xd5, + 0xe2, 0xd9, 0xe9, 0x12, 0xc8, 0x8c, 0xfc, 0x60, 0xa7, 0x42, 0xa0, 0x11, 0xfb, 0xdd, 0xd1, 0x88, + 0xda, 0x3b, 0x96, 0xe1, 0x58, 0xe6, 0x7c, 0x8e, 0xa9, 0x9d, 0x3d, 0xa1, 0x7b, 0x20, 0xba, 0x27, + 0xa6, 0x72, 0x88, 0x55, 0xdb, 0xdd, 0xc3, 0xaa, 0x4b, 0xd6, 0x67, 0xa0, 0x39, 0x4a, 0xee, 0x89, + 0x59, 0x0b, 0x91, 0xd7, 0x33, 0xd9, 0x69, 0x31, 0xbb, 0x9e, 0xc9, 0x66, 0xc5, 0x9c, 0xf4, 0x1f, + 0x04, 0x28, 0x7a, 0x7d, 0x23, 0xc9, 0x6e, 0x7c, 0x17, 0x44, 0xcb, 0xc4, 0x4a, 0xe7, 0x50, 0x75, + 0x30, 0x6f, 0x4b, 0xbe, 0x3a, 0x14, 0x2d, 0x13, 0xef, 0x10, 0x32, 0x6b, 0x19, 0xb4, 0x03, 0xb3, + 0x8e, 0xab, 0x1e, 0x18, 0xe6, 0x81, 0xe2, 0x6f, 0xf1, 0x53, 0xcb, 0x62, 0x4c, 0x24, 0x20, 0x72, + 0x6e, 0x9f, 0x1e, 0x31, 0x29, 0xfe, 0x58, 0x80, 0xd9, 0x15, 0xbd, 0x6d, 0x98, 0x8d, 0x4e, 0xcb, 0x48, 0x74, 0x83, 0xe1, 0x6d, 0xc8, 0x39, 0x44, 0x66, 0x30, 0x3b, 0x07, 0x70, 0x31, 0x4b, 0x53, - 0xc8, 0x34, 0xbd, 0x01, 0x25, 0x7c, 0xdc, 0x35, 0xd8, 0xb9, 0x02, 0x43, 0x39, 0x99, 0xf1, 0xeb, + 0xc8, 0x34, 0xbd, 0x09, 0x25, 0x7c, 0xd2, 0x31, 0xd8, 0xb9, 0x02, 0x43, 0x39, 0x99, 0xf1, 0xeb, 0x56, 0x0c, 0x78, 0x49, 0x12, 0xaf, 0xd3, 0xc7, 0x80, 0xc2, 0x55, 0x4a, 0x12, 0x68, 0x7c, 0x0c, - 0x73, 0x54, 0xf4, 0x8e, 0xe9, 0x24, 0xac, 0x2f, 0xe9, 0x57, 0xe1, 0x52, 0x54, 0x74, 0x92, 0xe5, - 0x7e, 0xc1, 0x5b, 0x79, 0x13, 0xdb, 0x89, 0x22, 0x54, 0x5f, 0xd7, 0x5c, 0x70, 0x92, 0x65, 0xfe, - 0x0d, 0x01, 0xde, 0xa0, 0xb2, 0xe9, 0xd1, 0xcb, 0x1e, 0xb6, 0x37, 0xb0, 0xea, 0x24, 0x0a, 0xaf, - 0x6f, 0xc2, 0x14, 0x83, 0xc9, 0xb4, 0x7f, 0x4e, 0x56, 0xf2, 0xc4, 0xcc, 0x68, 0xba, 0x96, 0x4d, - 0xcc, 0x0c, 0x9e, 0x24, 0xa9, 0xb0, 0x10, 0x57, 0x8a, 0x24, 0x6b, 0xfa, 0xb7, 0x05, 0x98, 0xe5, - 0x16, 0x1e, 0xe9, 0xca, 0xd5, 0x03, 0x62, 0xe0, 0xa0, 0x1a, 0xe4, 0x35, 0xfa, 0x4b, 0x71, 0x4f, - 0xba, 0x98, 0xca, 0x2f, 0x8e, 0x32, 0x0e, 0x19, 0x5b, 0xeb, 0xa4, 0x8b, 0x89, 0x85, 0xe9, 0xfd, - 0x26, 0x8a, 0x0a, 0x55, 0x72, 0xa4, 0x79, 0x49, 0xc7, 0x11, 0xcd, 0xeb, 0xd9, 0x69, 0x5c, 0x07, - 0xff, 0x28, 0xcd, 0x95, 0xc0, 0xde, 0xc1, 0xb3, 0x27, 0x6a, 0x50, 0x7c, 0x02, 0x57, 0x42, 0x5b, - 0xe7, 0xe1, 0x8a, 0xa7, 0x2e, 0x50, 0xf1, 0xd0, 0xf6, 0x7b, 0x40, 0x45, 0x1f, 0x43, 0x68, 0x83, - 0x5d, 0x61, 0x75, 0xf2, 0xa0, 0xca, 0x45, 0xd4, 0x31, 0x1b, 0x48, 0x61, 0x74, 0x07, 0x55, 0x21, - 0x8b, 0x8f, 0xbb, 0x8a, 0x8e, 0x1d, 0x8d, 0x4f, 0x5c, 0x52, 0x9c, 0x40, 0x52, 0x94, 0x01, 0xe3, - 0x7d, 0x1a, 0x1f, 0x77, 0x09, 0x11, 0xed, 0x90, 0x75, 0xd3, 0x5b, 0xd7, 0x69, 0xb1, 0x9d, 0xf3, - 0xb1, 0x40, 0xd0, 0x53, 0xb8, 0xb8, 0x92, 0xbf, 0xa4, 0x33, 0x11, 0xd2, 0x8f, 0x05, 0xb8, 0x16, - 0xdb, 0x6a, 0x49, 0x2e, 0x64, 0x1f, 0x42, 0x86, 0x56, 0x3e, 0x75, 0xc1, 0xca, 0x53, 0x2e, 0xe9, - 0xb3, 0x14, 0x1f, 0xe3, 0x32, 0x6e, 0x5b, 0x44, 0xb1, 0x89, 0x6f, 0xa1, 0x3d, 0x85, 0xc2, 0x91, - 0xe5, 0x62, 0xdb, 0x6f, 0xf6, 0xd4, 0x85, 0x9b, 0x7d, 0x86, 0x0a, 0xf0, 0x5a, 0xfc, 0x39, 0xcc, - 0x9a, 0x96, 0xa9, 0x44, 0x85, 0x5e, 0xbc, 0x2f, 0x95, 0x4c, 0xcb, 0x7c, 0x1e, 0x92, 0xeb, 0xcf, - 0x33, 0x7d, 0x9a, 0x48, 0x72, 0x9e, 0xf9, 0xa1, 0x00, 0x73, 0xbe, 0xa5, 0x93, 0xb0, 0xb9, 0xfb, - 0x2d, 0x48, 0x9b, 0xd6, 0xab, 0x8b, 0x6c, 0x51, 0x92, 0xfc, 0x64, 0xd5, 0x8b, 0x96, 0x28, 0xc9, - 0xfa, 0xfe, 0xcb, 0x14, 0xe4, 0x9e, 0x54, 0x93, 0xac, 0xe5, 0x87, 0x7c, 0xfb, 0x9b, 0xb5, 0x77, - 0x5c, 0x6f, 0xf7, 0xdf, 0x57, 0x7e, 0x52, 0x5d, 0xc7, 0x27, 0x5e, 0x6f, 0x27, 0x5c, 0x68, 0x05, - 0x72, 0xd1, 0x8d, 0xd2, 0x31, 0x35, 0x15, 0x70, 0x2d, 0x60, 0x98, 0xa4, 0x72, 0x3d, 0x57, 0x0b, - 0x21, 0xc6, 0xd5, 0x82, 0xbc, 0xc6, 0xb7, 0x14, 0x53, 0x17, 0x79, 0x4d, 0xc8, 0x44, 0x9c, 0x14, - 0xa7, 0xa4, 0x67, 0x00, 0xa4, 0x3a, 0x49, 0x36, 0xc9, 0x0f, 0xd2, 0x50, 0xdc, 0xee, 0x39, 0x07, - 0x09, 0xf7, 0xbe, 0x2a, 0x40, 0xb7, 0xe7, 0x1c, 0x90, 0x11, 0x79, 0x6c, 0xf2, 0x3a, 0x9f, 0xe3, - 0xc5, 0xe1, 0x55, 0x9a, 0xf1, 0xb5, 0x8e, 0x4d, 0x54, 0xe7, 0x42, 0xb0, 0x12, 0xb8, 0x82, 0xdc, - 0x1c, 0x85, 0x2c, 0x5b, 0xc7, 0xe6, 0x26, 0xf6, 0x21, 0x25, 0x93, 0x84, 0x89, 0xa4, 0x0f, 0x61, - 0x9a, 0x3c, 0x28, 0xae, 0x75, 0x91, 0x66, 0x9e, 0x22, 0x3c, 0x2d, 0x0b, 0x7d, 0x00, 0x39, 0xc6, - 0x4d, 0x56, 0xbf, 0x29, 0xba, 0xfa, 0xc5, 0xd5, 0x85, 0xab, 0x91, 0xae, 0x7b, 0x59, 0xca, 0x4a, - 0xd6, 0xba, 0x4b, 0x30, 0xb9, 0x67, 0xd9, 0x9a, 0x77, 0x98, 0xcb, 0x1e, 0x58, 0x7b, 0x32, 0x48, - 0xb3, 0x96, 0xc9, 0xe6, 0x44, 0x90, 0x7e, 0x4b, 0x80, 0x92, 0xdf, 0x10, 0x49, 0x2e, 0x08, 0xd5, - 0x88, 0x16, 0x2f, 0xde, 0x14, 0x44, 0x81, 0xd2, 0xbf, 0xa1, 0x16, 0x91, 0x66, 0x1d, 0xd1, 0x96, - 0x49, 0xb2, 0xa7, 0x7c, 0xc0, 0x1c, 0x7d, 0x52, 0x17, 0x6d, 0x5d, 0xea, 0xf3, 0xf3, 0x00, 0x2e, - 0x19, 0x1d, 0x32, 0x9f, 0x1b, 0x6e, 0xfb, 0x84, 0xc3, 0x36, 0x17, 0x7b, 0xa7, 0xc6, 0x73, 0x41, - 0x5a, 0xd5, 0x4b, 0x92, 0x7e, 0x8f, 0xee, 0x56, 0x07, 0x35, 0x49, 0x52, 0xd5, 0x0d, 0x28, 0xd8, - 0x4c, 0x34, 0x31, 0x6b, 0x2e, 0xa8, 0xed, 0x19, 0x9f, 0x95, 0x28, 0xfc, 0xb7, 0x53, 0x50, 0x7a, - 0xd6, 0xc3, 0xf6, 0xc9, 0xd7, 0x49, 0xdd, 0xb7, 0xa1, 0xf4, 0x4a, 0x35, 0x5c, 0x65, 0xcf, 0xb2, - 0x95, 0x5e, 0x57, 0x57, 0x5d, 0xcf, 0xdb, 0xa4, 0x40, 0xc8, 0x8f, 0x2d, 0x7b, 0x87, 0x12, 0x11, - 0x06, 0x74, 0x68, 0x5a, 0xaf, 0x4c, 0x85, 0x90, 0x29, 0x50, 0x3e, 0x36, 0xf9, 0x16, 0x72, 0xe5, - 0xdb, 0xff, 0xfe, 0x74, 0xe9, 0xd1, 0x58, 0x3e, 0x64, 0xd4, 0x5f, 0xae, 0xd7, 0x33, 0xf4, 0xf2, - 0xce, 0x4e, 0x63, 0x55, 0x16, 0xa9, 0xc8, 0x17, 0x4c, 0x62, 0xeb, 0xd8, 0x74, 0xa4, 0xbf, 0x93, - 0x02, 0x31, 0xd0, 0x51, 0x92, 0x0d, 0x59, 0x83, 0xfc, 0xcb, 0x1e, 0xb6, 0x8d, 0xd7, 0x68, 0x46, - 0xe0, 0x8c, 0x64, 0xda, 0xb9, 0x0f, 0xb3, 0xee, 0xb1, 0xa9, 0x30, 0x0f, 0x3f, 0xe6, 0xf8, 0xe1, - 0x39, 0x2c, 0x94, 0x5c, 0x52, 0x66, 0x42, 0xa7, 0x4e, 0x1f, 0x0e, 0xfa, 0x04, 0x66, 0x22, 0xda, - 0x4a, 0x7f, 0x39, 0x6d, 0xe5, 0x5f, 0x85, 0x14, 0xf5, 0x07, 0x02, 0x20, 0xaa, 0xa8, 0x06, 0xdb, - 0xe3, 0xff, 0xba, 0xf4, 0xa7, 0xbb, 0x20, 0x52, 0x7f, 0x4c, 0xc5, 0xd8, 0x53, 0x3a, 0x86, 0xe3, - 0x18, 0xe6, 0x3e, 0xef, 0x50, 0x45, 0x4a, 0x6f, 0xec, 0x6d, 0x32, 0xaa, 0xf4, 0x97, 0x60, 0x2e, - 0x52, 0x81, 0x24, 0x1b, 0xfb, 0x06, 0xcc, 0xec, 0xb1, 0x23, 0x58, 0x2a, 0x9c, 0x6f, 0x0f, 0xe6, - 0x29, 0x8d, 0xbd, 0x4f, 0xfa, 0xaf, 0x29, 0xb8, 0x24, 0x63, 0xc7, 0x6a, 0x1f, 0xe1, 0xe4, 0x55, - 0x58, 0x07, 0x7e, 0xf6, 0xa2, 0xbc, 0x96, 0x26, 0x73, 0x8c, 0x99, 0x2d, 0x73, 0xd1, 0x3d, 0xf6, - 0xb7, 0x47, 0xf7, 0xd8, 0xc1, 0x5d, 0x75, 0xbe, 0x53, 0x97, 0x89, 0xec, 0xd4, 0x59, 0x50, 0x62, - 0xa7, 0xc7, 0xba, 0xe2, 0xe0, 0x97, 0x66, 0xaf, 0xe3, 0x81, 0xa1, 0xf2, 0xa8, 0x42, 0x36, 0x18, - 0x4b, 0x13, 0xbf, 0xdc, 0xea, 0x75, 0xa8, 0xed, 0x5c, 0xb9, 0x42, 0xca, 0x7b, 0x76, 0xba, 0x54, - 0x8c, 0xa4, 0x39, 0x72, 0xd1, 0xf0, 0x9f, 0x89, 0x74, 0xe9, 0x7b, 0x70, 0xb9, 0x4f, 0xd9, 0x49, - 0x5a, 0x3c, 0xff, 0x2c, 0x0d, 0x6f, 0x44, 0xc5, 0x27, 0x0d, 0x71, 0xbe, 0xee, 0x0d, 0x5a, 0x87, - 0x42, 0xc7, 0x30, 0x5f, 0x6f, 0xf7, 0x72, 0xa6, 0x63, 0x98, 0x3e, 0x2d, 0xae, 0x6b, 0x4c, 0x7d, - 0xa5, 0x5d, 0x43, 0x85, 0x85, 0xb8, 0xb6, 0x4b, 0xb2, 0x7f, 0x7c, 0x26, 0xc0, 0x4c, 0xd2, 0xdb, - 0x72, 0xaf, 0xe7, 0x05, 0x27, 0xb5, 0xa0, 0xf0, 0x15, 0xec, 0xe3, 0xfd, 0xb6, 0x00, 0xa8, 0x65, - 0xf7, 0x4c, 0x02, 0x6a, 0x37, 0xac, 0xfd, 0x24, 0xab, 0x79, 0x09, 0x26, 0x0d, 0x53, 0xc7, 0xc7, - 0xb4, 0x9a, 0x19, 0x99, 0x3d, 0x44, 0x8e, 0x12, 0xd3, 0x63, 0x1d, 0x25, 0x4a, 0x9f, 0xc0, 0x5c, - 0xa4, 0x88, 0x49, 0xd6, 0xff, 0x4f, 0x52, 0x30, 0xc7, 0x2b, 0x92, 0xf8, 0x0e, 0xe6, 0x37, 0x61, - 0xb2, 0x4d, 0x64, 0x8e, 0x68, 0x67, 0xfa, 0x4e, 0xaf, 0x9d, 0x69, 0x66, 0xf4, 0x1d, 0x80, 0xae, - 0x8d, 0x8f, 0x14, 0xc6, 0x9a, 0x1e, 0x8b, 0x35, 0x47, 0x38, 0x28, 0x01, 0x7d, 0x2e, 0x40, 0x89, - 0x0c, 0xe8, 0xae, 0x6d, 0x75, 0x2d, 0x87, 0xd8, 0x2c, 0xce, 0x78, 0x30, 0xe7, 0xd9, 0xd9, 0xe9, - 0x52, 0x61, 0xd3, 0x30, 0xb7, 0x39, 0x63, 0xab, 0x39, 0xb6, 0x83, 0xbf, 0x17, 0xe6, 0x50, 0xae, - 0xb6, 0x2d, 0xed, 0x30, 0x38, 0x1c, 0x23, 0x33, 0x8b, 0x2f, 0xce, 0x91, 0x7e, 0x26, 0xc0, 0xa5, - 0xaf, 0x6c, 0xbb, 0xf8, 0xcf, 0x42, 0xd9, 0xd2, 0x73, 0x10, 0xe9, 0x8f, 0x86, 0xb9, 0x67, 0x25, - 0xb9, 0x71, 0xff, 0x7f, 0x04, 0x98, 0x0d, 0x09, 0x4e, 0xd2, 0xc0, 0x79, 0x5d, 0x3d, 0x15, 0x98, - 0x3b, 0x8c, 0x3b, 0x9e, 0xaa, 0xe4, 0x19, 0x9e, 0x9d, 0x75, 0xca, 0x32, 0xcc, 0x60, 0x32, 0x8b, - 0xd1, 0x2d, 0xde, 0x5d, 0x16, 0x64, 0xd2, 0xb7, 0xa3, 0x9f, 0xf7, 0x33, 0x54, 0x4e, 0xa4, 0x5f, - 0x25, 0x16, 0x56, 0x78, 0x50, 0x26, 0x39, 0xe4, 0xff, 0x69, 0x0a, 0xae, 0x54, 0xd9, 0x11, 0xb8, - 0xe7, 0x13, 0x92, 0x64, 0x47, 0x9c, 0x87, 0xe9, 0x23, 0x6c, 0x3b, 0x86, 0xc5, 0x56, 0xfb, 0x82, - 0xec, 0x3d, 0xa2, 0x05, 0xc8, 0x3a, 0xa6, 0xda, 0x75, 0x0e, 0x2c, 0xef, 0x38, 0xd1, 0x7f, 0xf6, - 0xfd, 0x57, 0x26, 0x5f, 0xdf, 0x7f, 0x65, 0x6a, 0xb4, 0xff, 0xca, 0xf4, 0x97, 0xf0, 0x5f, 0xe1, - 0x67, 0x77, 0xff, 0x56, 0x80, 0xab, 0x03, 0x9a, 0x4b, 0xb2, 0x73, 0x7e, 0x1f, 0xf2, 0x1a, 0x17, - 0x4c, 0xd6, 0x07, 0x76, 0x30, 0xd9, 0x20, 0xd9, 0x5e, 0x13, 0xfa, 0x9c, 0x9d, 0x2e, 0x81, 0x57, - 0xd4, 0xc6, 0x2a, 0x57, 0x0e, 0xf9, 0xad, 0x4b, 0xff, 0x0d, 0xa0, 0x54, 0x3b, 0x66, 0x9b, 0xf2, - 0x4d, 0x66, 0x95, 0xa0, 0xc7, 0x90, 0xed, 0xda, 0xd6, 0x91, 0xe1, 0x55, 0xa3, 0x18, 0x71, 0x5e, - 0xf0, 0xaa, 0xd1, 0xc7, 0xb5, 0xcd, 0x39, 0x64, 0x9f, 0x17, 0xb5, 0x20, 0xb7, 0x61, 0x69, 0x6a, - 0xfb, 0xb1, 0xd1, 0xf6, 0x06, 0xda, 0xbb, 0xe7, 0x0b, 0x2a, 0xfb, 0x3c, 0xdb, 0xaa, 0x7b, 0xe0, - 0x35, 0x82, 0x4f, 0x44, 0x0d, 0xc8, 0xd6, 0x5d, 0xb7, 0x4b, 0x12, 0xf9, 0xf8, 0xbb, 0x33, 0x86, - 0x50, 0xc2, 0xe2, 0x79, 0xdc, 0x7a, 0xec, 0xa8, 0x05, 0xb3, 0x4f, 0x68, 0xfc, 0x58, 0xb5, 0x6d, - 0xf5, 0xf4, 0xaa, 0x65, 0xee, 0x19, 0xfb, 0x7c, 0x99, 0xb8, 0x3d, 0x86, 0xcc, 0x27, 0xd5, 0xa6, - 0x3c, 0x28, 0x00, 0xad, 0x40, 0xb6, 0xf9, 0x88, 0x0b, 0x63, 0x66, 0xe4, 0xad, 0x31, 0x84, 0x35, - 0x1f, 0xc9, 0x3e, 0x1b, 0x5a, 0x83, 0xfc, 0xca, 0xa7, 0x3d, 0x1b, 0x73, 0x29, 0x53, 0x43, 0x3d, - 0x27, 0xfa, 0xa5, 0x50, 0x2e, 0x39, 0xcc, 0x8c, 0xbe, 0x07, 0x25, 0xa2, 0xb7, 0x96, 0xba, 0xdb, - 0xf6, 0xe4, 0x65, 0xa9, 0xbc, 0x6f, 0x8c, 0x21, 0xcf, 0xe7, 0xf4, 0x8e, 0x04, 0xfa, 0x44, 0x2d, - 0xc8, 0x50, 0x88, 0xb4, 0x17, 0x42, 0x90, 0xe9, 0x92, 0xa6, 0x11, 0xa8, 0x1b, 0x12, 0xfd, 0x8d, - 0xde, 0x81, 0x69, 0xd3, 0xd2, 0xb1, 0xd7, 0x99, 0x0b, 0x95, 0x4b, 0x67, 0xa7, 0x4b, 0x53, 0x5b, - 0x96, 0xce, 0x6c, 0x1d, 0xfe, 0x4b, 0x9e, 0x22, 0x99, 0x1a, 0xfa, 0xc2, 0x75, 0xc8, 0x90, 0x26, - 0x22, 0x73, 0xc8, 0xae, 0xea, 0xe0, 0x1d, 0xdb, 0xe0, 0xd2, 0xbc, 0xc7, 0x85, 0xbf, 0x9f, 0x82, - 0x54, 0xf3, 0x11, 0xb1, 0xe6, 0x77, 0x7b, 0xda, 0x21, 0x76, 0x79, 0x3a, 0x7f, 0xa2, 0x56, 0xbe, - 0x8d, 0xf7, 0x0c, 0x66, 0x74, 0xe5, 0x64, 0xfe, 0x84, 0xde, 0x02, 0x50, 0x35, 0x0d, 0x3b, 0x8e, - 0xe2, 0x85, 0x00, 0xe6, 0xe4, 0x1c, 0xa3, 0xac, 0xe3, 0x13, 0xc2, 0xe6, 0x60, 0xcd, 0xc6, 0xae, - 0xe7, 0x43, 0xc5, 0x9e, 0x08, 0x9b, 0x8b, 0x3b, 0x5d, 0xc5, 0xb5, 0x0e, 0xb1, 0x49, 0x9b, 0x34, - 0x47, 0x66, 0x85, 0x4e, 0xb7, 0x45, 0x08, 0x64, 0x42, 0xc3, 0xa6, 0x1e, 0xcc, 0x3e, 0x39, 0xd9, - 0x7f, 0x26, 0x22, 0x6d, 0xbc, 0x6f, 0xf0, 0x00, 0xba, 0x9c, 0xcc, 0x9f, 0x88, 0x96, 0xd4, 0x9e, - 0x7b, 0x40, 0x5b, 0x22, 0x27, 0xd3, 0xdf, 0xe8, 0x36, 0x94, 0x98, 0xdb, 0xa5, 0x82, 0x4d, 0x4d, - 0xa1, 0xf3, 0x60, 0x8e, 0x26, 0x17, 0x18, 0xb9, 0x66, 0x6a, 0x64, 0xd6, 0x43, 0x8f, 0x80, 0x13, - 0x94, 0xc3, 0x8e, 0x43, 0x74, 0x0a, 0x24, 0x57, 0xa5, 0x74, 0x76, 0xba, 0x94, 0x6f, 0xd2, 0x84, - 0xf5, 0xcd, 0x26, 0x59, 0x4b, 0x58, 0xae, 0xf5, 0x8e, 0xd3, 0xd0, 0x17, 0xfe, 0xa6, 0x00, 0xe9, - 0x27, 0xd5, 0xe6, 0x85, 0x55, 0xe6, 0x15, 0x34, 0x1d, 0x2a, 0xe8, 0x1d, 0x28, 0xed, 0x1a, 0xed, - 0xb6, 0x61, 0xee, 0x13, 0xfb, 0xea, 0xfb, 0x58, 0xf3, 0x14, 0x56, 0xe4, 0xe4, 0x6d, 0x46, 0x45, - 0xd7, 0x21, 0xaf, 0xd9, 0x58, 0xc7, 0xa6, 0x6b, 0xa8, 0x6d, 0x87, 0x6b, 0x2e, 0x4c, 0x5a, 0xf8, - 0xcb, 0x02, 0x4c, 0xd2, 0xce, 0x8a, 0xde, 0x84, 0x9c, 0x66, 0x99, 0xae, 0x6a, 0x98, 0x7c, 0xd6, - 0xc9, 0xc9, 0x01, 0x61, 0x68, 0xf1, 0x6e, 0xc0, 0x8c, 0xaa, 0x69, 0x56, 0xcf, 0x74, 0x15, 0x53, - 0xed, 0x60, 0x5e, 0xcc, 0x3c, 0xa7, 0x6d, 0xa9, 0x1d, 0x8c, 0x96, 0xc0, 0x7b, 0xf4, 0x23, 0x3b, - 0x73, 0x32, 0x70, 0xd2, 0x3a, 0x3e, 0x59, 0xc0, 0x90, 0xf3, 0x7b, 0x35, 0xa9, 0x6f, 0xcf, 0xf1, - 0x4b, 0x40, 0x7f, 0xa3, 0x77, 0xe1, 0xd2, 0xcb, 0x9e, 0xda, 0x36, 0xf6, 0xe8, 0xe6, 0x17, 0xf5, - 0x52, 0xa7, 0x2f, 0x63, 0x45, 0x41, 0x7e, 0x1a, 0x95, 0x40, 0xdf, 0xe9, 0x0d, 0x82, 0x74, 0x30, - 0x08, 0x98, 0xcb, 0x8e, 0x74, 0x02, 0xb3, 0x32, 0x76, 0xed, 0x93, 0x16, 0x0b, 0x76, 0xad, 0x1d, - 0x61, 0xd3, 0x25, 0x75, 0xb7, 0xba, 0x98, 0x39, 0x89, 0x78, 0x75, 0xf7, 0x09, 0xe8, 0x16, 0x14, - 0x55, 0x97, 0x74, 0x37, 0x57, 0x31, 0x7b, 0x9d, 0x5d, 0x6c, 0x33, 0x57, 0x00, 0xb9, 0xc0, 0xa9, - 0x5b, 0x94, 0xc8, 0x23, 0x32, 0xec, 0x13, 0x85, 0xee, 0x13, 0xf1, 0x57, 0x03, 0x25, 0xd5, 0x08, - 0x45, 0xba, 0x07, 0x97, 0x49, 0x3d, 0x6b, 0xa6, 0x66, 0x9f, 0x74, 0x89, 0xe4, 0xa7, 0xf4, 0xaf, - 0x83, 0xc4, 0xd0, 0x39, 0x0d, 0x3d, 0x9e, 0x91, 0x7e, 0x3a, 0x05, 0x85, 0xda, 0x71, 0xd7, 0xb2, - 0x13, 0xdd, 0xd5, 0xa9, 0xc0, 0x34, 0x07, 0xbe, 0x23, 0x8e, 0x62, 0xfb, 0x66, 0x20, 0xef, 0x1c, - 0x9a, 0x33, 0xa2, 0x0a, 0x00, 0x73, 0xa8, 0xa4, 0x7e, 0x38, 0xe9, 0x0b, 0x9c, 0x1c, 0x51, 0x36, - 0x1a, 0x6c, 0xb0, 0x05, 0xf9, 0xce, 0x91, 0xa6, 0x29, 0x7b, 0x46, 0xdb, 0xe5, 0x7e, 0x69, 0xf1, - 0x2e, 0xd4, 0x9b, 0xcf, 0xab, 0xd5, 0xc7, 0x34, 0x13, 0xf3, 0xe7, 0x0a, 0x9e, 0x65, 0x20, 0x12, - 0xd8, 0x6f, 0xf4, 0x0d, 0xe0, 0x81, 0x2f, 0x8a, 0xe3, 0x85, 0xb1, 0x55, 0x0a, 0x67, 0xa7, 0x4b, - 0x39, 0x99, 0x52, 0x9b, 0xcd, 0x96, 0x9c, 0x63, 0x19, 0x9a, 0x8e, 0x7b, 0x91, 0x50, 0x87, 0xe9, - 0xf1, 0x43, 0x1d, 0xfe, 0x9a, 0x00, 0x57, 0xb8, 0x8e, 0x94, 0x5d, 0xea, 0xde, 0xad, 0xb6, 0x0d, - 0xf7, 0x44, 0x39, 0x3c, 0x9a, 0xcf, 0x52, 0x93, 0xe7, 0x97, 0x63, 0x75, 0x1d, 0x6a, 0xe2, 0xb2, - 0xa7, 0xf1, 0x93, 0x0d, 0xce, 0xbc, 0x7e, 0x54, 0x33, 0x5d, 0xfb, 0xa4, 0x72, 0xf5, 0xec, 0x74, - 0x69, 0x6e, 0x30, 0xf5, 0xb9, 0x3c, 0xe7, 0x0c, 0xb2, 0xa0, 0x3a, 0x00, 0xf6, 0xbb, 0x18, 0x9d, - 0xc1, 0xe2, 0x97, 0xae, 0xd8, 0xbe, 0x28, 0x87, 0x78, 0xd1, 0x5d, 0x10, 0x79, 0x68, 0xc9, 0x9e, - 0xd1, 0xc6, 0x8a, 0x63, 0x7c, 0x8a, 0xe9, 0x5c, 0x97, 0x96, 0x8b, 0x8c, 0x4e, 0x44, 0x34, 0x8d, - 0x4f, 0x31, 0x7a, 0x00, 0x97, 0x83, 0x16, 0x50, 0x76, 0x71, 0xdb, 0x7a, 0xc5, 0xb2, 0xe7, 0x69, - 0x76, 0xe4, 0x6b, 0xbf, 0x42, 0x92, 0x08, 0xcb, 0xc2, 0xf7, 0x61, 0x7e, 0x58, 0x85, 0xc3, 0x03, - 0x22, 0xc7, 0xce, 0x2b, 0xdf, 0x8b, 0x6e, 0x56, 0x8c, 0xd1, 0x71, 0xf9, 0x86, 0xc5, 0xfb, 0xa9, - 0xf7, 0x04, 0xe9, 0xef, 0xa5, 0xa0, 0x50, 0xe9, 0xb5, 0x0f, 0x9f, 0x76, 0x9b, 0x2c, 0x2c, 0x1f, - 0x5d, 0x83, 0x9c, 0xae, 0xba, 0x2a, 0x2b, 0xa4, 0xc0, 0x42, 0xcc, 0x08, 0x81, 0xd6, 0xe6, 0x0e, - 0x94, 0x42, 0xbe, 0x20, 0xdc, 0xe3, 0x9d, 0x56, 0x3b, 0x20, 0x53, 0xa7, 0xf4, 0xf7, 0x60, 0x3e, - 0x94, 0x91, 0xee, 0x2c, 0x28, 0xd8, 0x74, 0x6d, 0x03, 0xb3, 0xdd, 0xb1, 0xb4, 0x1c, 0x72, 0x58, - 0x69, 0x90, 0xe4, 0x1a, 0x4b, 0x45, 0x2d, 0x98, 0x21, 0x19, 0x4f, 0x14, 0x3a, 0x0b, 0x7a, 0xbb, - 0x97, 0x0f, 0x62, 0xaa, 0x15, 0x29, 0x77, 0x99, 0xea, 0xa7, 0x4a, 0x79, 0xe8, 0x4f, 0x39, 0x8f, - 0x03, 0xca, 0xc2, 0x47, 0x20, 0xf6, 0x67, 0x08, 0xeb, 0x32, 0xc3, 0x74, 0x79, 0x29, 0xac, 0xcb, - 0x74, 0x48, 0x4f, 0x6b, 0x99, 0x6c, 0x46, 0x9c, 0x94, 0x7e, 0x96, 0x86, 0xa2, 0xd7, 0x33, 0x93, - 0x34, 0xab, 0x2b, 0x30, 0x49, 0xfa, 0x91, 0xe7, 0x5e, 0x71, 0x7b, 0xc4, 0x80, 0xe0, 0x3e, 0xd6, - 0xa4, 0x7f, 0x79, 0x08, 0x90, 0xb2, 0x26, 0x31, 0xfd, 0x2c, 0xfc, 0x77, 0x01, 0x32, 0xd4, 0x92, - 0x7d, 0x00, 0x19, 0x1a, 0x97, 0x2f, 0x8c, 0x8c, 0xcb, 0xf7, 0x8e, 0xe7, 0x49, 0x56, 0x7f, 0x61, - 0x49, 0x85, 0xac, 0xab, 0x0a, 0xf5, 0xef, 0xb1, 0x6c, 0x17, 0xeb, 0xdc, 0x52, 0xbc, 0x7e, 0x5e, - 0x3b, 0x7a, 0x96, 0xb0, 0xc7, 0x87, 0xde, 0x80, 0x34, 0x99, 0xbb, 0xa6, 0xd9, 0x51, 0xfd, 0xd9, - 0xe9, 0x52, 0x9a, 0xcc, 0x5a, 0x84, 0x86, 0x96, 0x21, 0x1f, 0x9d, 0x4d, 0x88, 0xb1, 0x41, 0xa7, - 0xc3, 0xd0, 0x4c, 0x00, 0x6d, 0x7f, 0x08, 0x31, 0x94, 0xc4, 0xda, 0x92, 0x1f, 0xd2, 0xff, 0x86, - 0xc0, 0x7d, 0x12, 0x9b, 0x1a, 0x59, 0xb3, 0xec, 0x24, 0x17, 0x95, 0x7b, 0x20, 0xda, 0xaa, 0xa9, - 0x5b, 0x1d, 0xe3, 0x53, 0xcc, 0x50, 0xb9, 0xc3, 0x8f, 0x2b, 0x4a, 0x3e, 0x9d, 0xc2, 0x67, 0x47, - 0xfa, 0x2f, 0x02, 0xf7, 0x5f, 0xf4, 0x8b, 0x91, 0xec, 0xa1, 0x72, 0x9e, 0x6f, 0xe9, 0x99, 0x7b, - 0x96, 0xe7, 0x7e, 0xf1, 0xe6, 0x30, 0x67, 0xa3, 0x86, 0xb9, 0x67, 0x79, 0xc7, 0x63, 0xb6, 0x47, - 0x70, 0x16, 0x7e, 0x05, 0x26, 0x69, 0xf2, 0x6b, 0xf4, 0x0d, 0xdf, 0x67, 0x36, 0x25, 0xa6, 0xa5, - 0x3f, 0x4e, 0xc1, 0xdb, 0xb4, 0xaa, 0xcf, 0xb1, 0x6d, 0xec, 0x9d, 0x6c, 0xdb, 0x96, 0x8b, 0x35, - 0x17, 0xeb, 0xc1, 0xae, 0x54, 0x82, 0x4d, 0xa0, 0x43, 0x8e, 0x9f, 0xe7, 0x19, 0x3a, 0xbf, 0x39, - 0xe3, 0xc9, 0x97, 0x43, 0xab, 0x59, 0x76, 0x0e, 0xd8, 0x58, 0x95, 0xb3, 0x4c, 0x72, 0x43, 0x47, - 0x2b, 0x90, 0xeb, 0x7a, 0xd5, 0xb8, 0x90, 0xcb, 0x88, 0xcf, 0x85, 0xd6, 0xa1, 0xc4, 0x0b, 0xaa, - 0xb6, 0x8d, 0x23, 0xac, 0xa8, 0xee, 0x45, 0x86, 0x70, 0x81, 0xf1, 0xae, 0x10, 0xd6, 0x15, 0x57, - 0xfa, 0x1b, 0x19, 0xb8, 0x75, 0x8e, 0x8a, 0x93, 0xec, 0x5e, 0x0b, 0x90, 0x3d, 0x22, 0x2f, 0x32, - 0x78, 0xed, 0xb3, 0xb2, 0xff, 0x8c, 0x76, 0x23, 0xeb, 0xc0, 0x9e, 0x6a, 0xb4, 0xc9, 0xba, 0xc1, - 0x9c, 0xf4, 0x86, 0xbb, 0x01, 0xc5, 0x3b, 0xbd, 0x85, 0x56, 0x8c, 0xc7, 0x54, 0x10, 0xcd, 0xe6, - 0xa0, 0xcf, 0x04, 0x58, 0x60, 0x2f, 0x64, 0x9e, 0x62, 0x7d, 0xaf, 0xc9, 0xd0, 0xd7, 0xac, 0xc6, - 0xbc, 0x66, 0x2c, 0x1d, 0x95, 0x43, 0xef, 0xe2, 0x05, 0x99, 0x0f, 0xbf, 0x2d, 0x5c, 0x94, 0x85, - 0xdf, 0x14, 0x20, 0x1f, 0x22, 0xa0, 0xdb, 0x03, 0x71, 0x39, 0xf9, 0xb3, 0xb8, 0x60, 0x9c, 0x5b, - 0x03, 0xc1, 0x38, 0x95, 0xec, 0x17, 0xa7, 0x4b, 0x19, 0x99, 0xf9, 0x7b, 0x7b, 0x61, 0x39, 0x37, - 0x82, 0x6b, 0x60, 0xd2, 0x7d, 0x99, 0xbc, 0x7b, 0x60, 0x28, 0xac, 0x53, 0xbd, 0x63, 0x24, 0x0a, - 0xeb, 0xc8, 0x93, 0xf4, 0xa3, 0x14, 0xcc, 0xae, 0xe8, 0x7a, 0xb3, 0x49, 0xa1, 0x40, 0x92, 0x63, - 0x0c, 0x41, 0x86, 0xd8, 0x07, 0x3c, 0x86, 0x88, 0xfe, 0x46, 0xef, 0x00, 0xd2, 0x0d, 0x87, 0x5d, - 0xa7, 0xe0, 0x1c, 0xa8, 0xba, 0xf5, 0x2a, 0x38, 0x2d, 0x9e, 0xf5, 0x52, 0x9a, 0x5e, 0x02, 0x6a, - 0x02, 0x35, 0x5a, 0x15, 0xc7, 0x55, 0xfd, 0xdd, 0xf0, 0x5b, 0x63, 0x45, 0xa5, 0x30, 0x6b, 0xd6, - 0x7f, 0x94, 0x73, 0x44, 0x0e, 0xfd, 0x49, 0x6c, 0x34, 0x83, 0x34, 0x8a, 0xab, 0xa8, 0x8e, 0x17, - 0x4f, 0xc1, 0x2e, 0x72, 0x28, 0x32, 0xfa, 0x8a, 0xc3, 0xc2, 0x24, 0x98, 0x1b, 0x76, 0xa0, 0x9a, - 0x24, 0xf7, 0x32, 0x7f, 0x47, 0x80, 0xa2, 0x8c, 0xf7, 0x6c, 0xec, 0x1c, 0x24, 0xa9, 0xf3, 0xc7, - 0x30, 0x63, 0x33, 0xa9, 0xca, 0x9e, 0x6d, 0x75, 0x2e, 0x32, 0x57, 0xe4, 0x39, 0xe3, 0x63, 0xdb, - 0xea, 0xf0, 0x29, 0xf9, 0x39, 0x94, 0xfc, 0x32, 0x26, 0x59, 0xf9, 0xbf, 0x4b, 0x23, 0x2e, 0x99, - 0xe0, 0xa4, 0x8f, 0x6d, 0x93, 0xd5, 0x00, 0xdd, 0xcf, 0x0e, 0x17, 0x34, 0x49, 0x35, 0xfc, 0x67, - 0x01, 0x8a, 0xcd, 0xde, 0x2e, 0xbb, 0x26, 0x28, 0x39, 0x0d, 0xd4, 0x20, 0xd7, 0xc6, 0x7b, 0xae, - 0xf2, 0x5a, 0x0e, 0xc4, 0x59, 0xc2, 0x4a, 0xdd, 0xa7, 0x9f, 0x00, 0xd8, 0x34, 0xe4, 0x88, 0xca, - 0x49, 0x5f, 0x50, 0x4e, 0x8e, 0xf2, 0x12, 0x32, 0x59, 0x75, 0x4a, 0x7e, 0x35, 0x93, 0x5c, 0x5f, - 0x5e, 0x44, 0x66, 0x87, 0xf4, 0x45, 0x66, 0x87, 0x59, 0x7e, 0x52, 0x1d, 0x3f, 0x43, 0x94, 0x61, - 0x8e, 0x9a, 0x65, 0x8a, 0xda, 0xed, 0xb6, 0x0d, 0x0f, 0xa7, 0xd0, 0xf9, 0x27, 0x23, 0xcf, 0xd2, - 0xa4, 0x15, 0x96, 0x42, 0x11, 0x0a, 0xfa, 0x81, 0x00, 0x33, 0x7b, 0x36, 0xc6, 0x9f, 0x62, 0x85, - 0x4e, 0xc9, 0xe3, 0x1d, 0xc5, 0xaf, 0x92, 0x32, 0x7c, 0xe9, 0xa3, 0xba, 0x3c, 0x7b, 0x71, 0x93, - 0xbc, 0x17, 0x6d, 0x81, 0xa8, 0xb5, 0xd9, 0xe1, 0xa1, 0xef, 0x16, 0x30, 0x35, 0xfe, 0x00, 0x28, - 0x31, 0xe6, 0xc0, 0x33, 0xe0, 0x19, 0x19, 0x4c, 0xaa, 0xae, 0xf0, 0xab, 0xd9, 0xa8, 0xb1, 0x1d, - 0x75, 0x0b, 0x08, 0x87, 0x5e, 0x87, 0x6e, 0x74, 0x2b, 0xcb, 0x58, 0xd5, 0xb9, 0xe5, 0x4e, 0xc6, - 0x95, 0xff, 0xc0, 0xc7, 0xd5, 0x0b, 0x98, 0xa5, 0xfd, 0x26, 0xe9, 0x08, 0x4a, 0xe9, 0x1f, 0xa6, - 0x01, 0x85, 0x25, 0x7f, 0x75, 0xfd, 0x2d, 0x95, 0x5c, 0x7f, 0x5b, 0x03, 0x29, 0x64, 0x0c, 0xb5, - 0x55, 0xc7, 0x55, 0x98, 0xff, 0x99, 0xa3, 0x74, 0xb1, 0xad, 0x38, 0x58, 0xb3, 0xf8, 0x25, 0x3a, - 0x82, 0xbc, 0x18, 0xe4, 0xdc, 0x50, 0x1d, 0xf7, 0x19, 0xcb, 0xb7, 0x8d, 0xed, 0x26, 0xcd, 0x85, - 0x1e, 0xc1, 0x95, 0x8e, 0x7a, 0x1c, 0xc7, 0x3f, 0x49, 0xf9, 0xe7, 0x3a, 0xea, 0xf1, 0x00, 0xd3, - 0xfb, 0xb0, 0x10, 0xcf, 0xa4, 0x38, 0xd8, 0x3b, 0x9f, 0xba, 0x12, 0xc3, 0xd8, 0xc4, 0x2e, 0x5a, - 0x01, 0x08, 0x40, 0x04, 0x5f, 0xa3, 0xc7, 0xc1, 0x10, 0x39, 0x1f, 0x43, 0x48, 0x3f, 0x14, 0xa0, - 0xb8, 0x69, 0xec, 0xdb, 0x6a, 0xa2, 0x57, 0xd4, 0xa0, 0xf7, 0xa3, 0x07, 0x7a, 0xf9, 0x87, 0x0b, - 0x71, 0x0e, 0x1b, 0x2c, 0x87, 0xb7, 0x69, 0xc7, 0x19, 0xc8, 0xd2, 0xe7, 0x97, 0x28, 0xc9, 0x39, - 0x5f, 0x83, 0xb7, 0xa8, 0x0b, 0x1c, 0xf7, 0x7f, 0xf9, 0x4a, 0xd0, 0x8d, 0xf4, 0xfb, 0x02, 0x2c, - 0x0e, 0x7b, 0x4b, 0x92, 0x03, 0x42, 0xa6, 0x97, 0xf5, 0xd1, 0x37, 0x28, 0xfe, 0x88, 0x38, 0x67, - 0xa6, 0x41, 0x7c, 0x24, 0x80, 0x5f, 0xb6, 0x26, 0xbd, 0xc6, 0x8f, 0xfd, 0x76, 0xa4, 0x7f, 0xbd, - 0x00, 0x33, 0xbc, 0x7e, 0x3b, 0xa6, 0x61, 0x99, 0xe8, 0x01, 0xa4, 0xf7, 0xf9, 0xde, 0x7f, 0x3e, - 0x76, 0xcb, 0x33, 0xb8, 0x21, 0xaf, 0x3e, 0x21, 0x93, 0xbc, 0x84, 0xa5, 0xdb, 0x73, 0x63, 0xca, - 0x13, 0x38, 0x79, 0x87, 0x59, 0xba, 0x3d, 0x17, 0x35, 0xa1, 0xa4, 0x05, 0xd7, 0x72, 0x29, 0x84, - 0x3d, 0x3d, 0x74, 0x1f, 0x30, 0xf6, 0x82, 0xb4, 0xfa, 0x84, 0x5c, 0xd4, 0x22, 0x09, 0xa8, 0x1a, - 0xbe, 0x0d, 0x2a, 0x33, 0xe0, 0x41, 0x16, 0xc4, 0x12, 0x47, 0x6f, 0xa2, 0xaa, 0x4f, 0x84, 0x2e, - 0x8d, 0x42, 0xef, 0xc3, 0x94, 0x4e, 0xef, 0x1d, 0xe2, 0xab, 0x4a, 0x5c, 0x87, 0x88, 0x5c, 0xef, - 0x54, 0x9f, 0x90, 0x39, 0x07, 0x5a, 0x83, 0x19, 0xf6, 0x8b, 0x61, 0x1e, 0xbe, 0x16, 0xdc, 0x1a, - 0x2e, 0x21, 0x64, 0x8d, 0xd5, 0x27, 0xe4, 0xbc, 0x1e, 0x50, 0xd1, 0x13, 0xc8, 0x6b, 0x6d, 0xac, - 0xda, 0x5c, 0xd4, 0xed, 0xa1, 0x61, 0x6f, 0x03, 0x77, 0x15, 0xd5, 0x27, 0x64, 0xd0, 0x7c, 0x22, - 0x29, 0x94, 0x4d, 0xaf, 0xac, 0xe1, 0x92, 0xde, 0x1d, 0x5a, 0xa8, 0xc1, 0xfb, 0x7f, 0xea, 0xd4, - 0x4a, 0xf3, 0xa9, 0xe8, 0x9b, 0x90, 0x71, 0x34, 0xd5, 0xe4, 0x0b, 0xd3, 0xe2, 0x90, 0x3b, 0x45, - 0x02, 0x66, 0x9a, 0x1b, 0x7d, 0xc0, 0xe0, 0x92, 0x7b, 0xec, 0x6d, 0xf6, 0xc6, 0xe9, 0x34, 0x12, - 0xbb, 0x4e, 0x74, 0x8a, 0x29, 0x81, 0xe8, 0x41, 0x25, 0xf8, 0x50, 0xa1, 0x01, 0xa5, 0x74, 0x77, - 0x37, 0x5e, 0x0f, 0x03, 0x01, 0xc0, 0x75, 0x1a, 0x20, 0xef, 0x11, 0xd1, 0x26, 0x14, 0x98, 0xa0, - 0x1e, 0x8b, 0x4d, 0x9d, 0x5f, 0x1e, 0x7a, 0x8c, 0x1b, 0x13, 0x1d, 0x5b, 0x9f, 0x90, 0x67, 0xd4, - 0x10, 0x39, 0x28, 0x57, 0x07, 0xdb, 0xfb, 0x6c, 0x1b, 0x79, 0x44, 0xb9, 0xc2, 0xbe, 0x71, 0x7e, - 0xb9, 0x28, 0x11, 0xfd, 0x3a, 0x5c, 0x62, 0x82, 0x5c, 0xee, 0xf2, 0xc3, 0x3d, 0x47, 0xde, 0x1a, - 0x7a, 0x04, 0x3b, 0x34, 0x9e, 0xb4, 0x3e, 0x21, 0x23, 0x75, 0x20, 0x11, 0x69, 0x70, 0x99, 0xbd, - 0x81, 0x07, 0x24, 0xda, 0x3c, 0x86, 0x6e, 0xfe, 0x26, 0x7d, 0xc5, 0x3b, 0xc3, 0x5e, 0x11, 0x1b, - 0x27, 0x59, 0x9f, 0x90, 0xe7, 0xd4, 0xc1, 0xd4, 0xa0, 0x1a, 0x36, 0x0f, 0xfd, 0xe2, 0xdd, 0xed, - 0x9d, 0xd1, 0xd5, 0x88, 0x0b, 0x99, 0xf3, 0xab, 0x11, 0x49, 0x24, 0x0d, 0xe8, 0x07, 0xbe, 0xd3, - 0xce, 0x34, 0x33, 0xb4, 0x01, 0x63, 0xe2, 0xc3, 0x48, 0x03, 0x1e, 0x84, 0xc8, 0xa8, 0x0c, 0xa9, - 0x7d, 0x6d, 0xbe, 0x30, 0x74, 0x01, 0xf5, 0x63, 0xa0, 0xea, 0x13, 0x72, 0x6a, 0x5f, 0x43, 0x1f, - 0x41, 0x96, 0x05, 0xb4, 0x1c, 0x9b, 0xf3, 0xc5, 0xa1, 0x93, 0x78, 0x34, 0x2c, 0xa8, 0x3e, 0x21, - 0xd3, 0x18, 0x1a, 0xde, 0x91, 0x79, 0xb0, 0x02, 0x15, 0x51, 0x1e, 0x11, 0xc7, 0xda, 0x17, 0x32, - 0x42, 0x3a, 0x8c, 0xed, 0x13, 0xd1, 0x36, 0x14, 0xf9, 0x04, 0xee, 0x39, 0x5f, 0x8b, 0x43, 0x9d, - 0x1c, 0xe2, 0xfc, 0xaf, 0xeb, 0x74, 0xa3, 0x2a, 0x44, 0x27, 0x6d, 0x17, 0x95, 0xc8, 0xdb, 0x6e, - 0x76, 0x68, 0xdb, 0x0d, 0xf5, 0x05, 0x26, 0x6d, 0x67, 0x0f, 0x24, 0xa2, 0x6f, 0xc3, 0x24, 0x1b, - 0x27, 0x88, 0x8a, 0x8c, 0xf3, 0xdb, 0xe9, 0x1b, 0x22, 0x2c, 0x3f, 0x99, 0xbd, 0x5c, 0xee, 0xd3, - 0xa8, 0xb4, 0xad, 0xfd, 0xf9, 0xb9, 0xa1, 0xb3, 0xd7, 0xa0, 0x77, 0x26, 0x99, 0xbd, 0xdc, 0x80, - 0x4a, 0x3a, 0x90, 0xcd, 0x52, 0xf8, 0x10, 0xbb, 0x34, 0xb4, 0x03, 0xc5, 0xb8, 0x3a, 0xd6, 0x69, - 0xb4, 0x49, 0x40, 0xf6, 0x27, 0x56, 0x07, 0x2b, 0x74, 0x52, 0xbc, 0x3c, 0x7a, 0x62, 0x8d, 0x5c, - 0xf4, 0xe4, 0x4f, 0xac, 0x8c, 0x8a, 0x9e, 0x83, 0xc8, 0x6f, 0x1b, 0x51, 0x3c, 0xd7, 0x9b, 0xf9, - 0x2b, 0x54, 0xde, 0xbd, 0xd8, 0x05, 0x31, 0xce, 0x2b, 0xab, 0x4e, 0x10, 0x45, 0x34, 0x05, 0x7d, - 0x0c, 0xb3, 0x54, 0x9e, 0xa2, 0x05, 0x17, 0xc4, 0xcc, 0xcf, 0x0f, 0x5c, 0x37, 0x32, 0xfc, 0x2e, - 0x19, 0x4f, 0xb2, 0xa8, 0xf5, 0x25, 0x91, 0xf1, 0x60, 0x98, 0x86, 0x4b, 0xd7, 0xee, 0x85, 0xa1, - 0xe3, 0x21, 0x7a, 0x39, 0x26, 0x19, 0x0f, 0x06, 0xa3, 0x90, 0x6e, 0xdc, 0x37, 0xe3, 0xbd, 0x39, - 0xb4, 0x1b, 0x0f, 0x99, 0xec, 0x0a, 0x6e, 0x64, 0x9e, 0x5b, 0x05, 0x60, 0x38, 0x92, 0x9a, 0xc6, - 0x8b, 0x43, 0x0d, 0x80, 0x7e, 0x57, 0x44, 0x62, 0x00, 0xb4, 0x3d, 0x1a, 0x31, 0x00, 0xd8, 0xa1, - 0xc7, 0xfc, 0xf5, 0xe1, 0x8b, 0x55, 0xf8, 0x58, 0x94, 0x2e, 0x56, 0x94, 0x80, 0x56, 0x20, 0x47, - 0x8c, 0xfa, 0x13, 0x3a, 0xc2, 0x6f, 0x0c, 0xc5, 0xf0, 0x7d, 0x31, 0x4a, 0xf5, 0x09, 0x39, 0xfb, - 0x92, 0x93, 0x48, 0xaf, 0x62, 0x22, 0xf8, 0xd8, 0xbe, 0x3f, 0xb4, 0x57, 0x0d, 0x06, 0xa7, 0x90, - 0x5e, 0xf5, 0x32, 0xa0, 0x06, 0x4b, 0x9e, 0xc3, 0x8e, 0x33, 0xe6, 0xdf, 0x1e, 0xbd, 0xe4, 0x45, - 0x0f, 0x5f, 0xfc, 0x25, 0x8f, 0x93, 0xd9, 0x92, 0xa7, 0x2b, 0x8e, 0x43, 0x3d, 0x1c, 0xe6, 0x6f, - 0x8d, 0x58, 0xf2, 0xfa, 0x36, 0x38, 0xd9, 0x92, 0xa7, 0x37, 0x19, 0x27, 0xb1, 0xfe, 0x6c, 0xef, - 0x76, 0x1d, 0x0e, 0xef, 0xee, 0x0c, 0xb5, 0xfe, 0x62, 0xaf, 0xff, 0x21, 0xd6, 0x9f, 0x1d, 0x49, - 0x40, 0xdf, 0x81, 0x69, 0xbe, 0xa1, 0x34, 0x7f, 0x77, 0x84, 0x8d, 0x1d, 0xde, 0x03, 0x24, 0xdd, - 0x91, 0xf3, 0xb0, 0xc9, 0x81, 0x6d, 0x64, 0xb1, 0xc9, 0xef, 0xde, 0x88, 0xc9, 0x61, 0x60, 0x2f, - 0x8d, 0x4d, 0x0e, 0x01, 0x99, 0x94, 0xc6, 0x61, 0x9b, 0x30, 0xf3, 0xbf, 0x34, 0xb4, 0x34, 0xd1, - 0xdd, 0x28, 0x52, 0x1a, 0xce, 0x43, 0x17, 0x0b, 0xba, 0x56, 0x33, 0xed, 0x7c, 0x63, 0xf8, 0x62, - 0xd1, 0x0f, 0xeb, 0xeb, 0xde, 0x71, 0x11, 0xd3, 0xca, 0x5f, 0x11, 0xe0, 0x3a, 0xeb, 0x03, 0x74, - 0xb3, 0xfc, 0x44, 0xf1, 0xcf, 0x3a, 0x42, 0x7b, 0x16, 0x0f, 0xa8, 0xf8, 0x6f, 0x5f, 0x7c, 0x6b, - 0xde, 0x7b, 0xe3, 0x5b, 0xea, 0xa8, 0x7c, 0x44, 0x19, 0x1d, 0x86, 0xee, 0xe6, 0x1f, 0x0e, 0x55, - 0x46, 0x14, 0x91, 0x12, 0x65, 0x70, 0x1e, 0xd4, 0x86, 0x79, 0x36, 0x24, 0x02, 0xf4, 0xe3, 0x17, - 0xfd, 0xd1, 0x50, 0xa7, 0xc1, 0x91, 0xb8, 0xaf, 0x3e, 0x21, 0x5f, 0x79, 0x19, 0x9b, 0xa1, 0x32, - 0xcd, 0x0f, 0x9f, 0xfd, 0x40, 0xd3, 0x92, 0x28, 0xae, 0x65, 0xb2, 0x57, 0xc5, 0xf9, 0xb5, 0x4c, - 0xf6, 0x0d, 0x71, 0x61, 0x2d, 0x93, 0xbd, 0x26, 0xbe, 0xb9, 0x96, 0xc9, 0x2e, 0x89, 0xd7, 0xd7, - 0x32, 0x59, 0x49, 0xbc, 0x29, 0xfd, 0xce, 0x02, 0x14, 0x3c, 0xf4, 0xc6, 0x50, 0xd4, 0xc3, 0x30, - 0x8a, 0x5a, 0x1c, 0x86, 0xa2, 0x38, 0xde, 0xe3, 0x30, 0xea, 0x61, 0x18, 0x46, 0x2d, 0x0e, 0x83, - 0x51, 0x01, 0x0f, 0xc1, 0x51, 0xad, 0x61, 0x38, 0xea, 0xde, 0x18, 0x38, 0xca, 0x17, 0xd5, 0x0f, - 0xa4, 0x56, 0x07, 0x81, 0xd4, 0xdb, 0xa3, 0x81, 0x94, 0x2f, 0x2a, 0x84, 0xa4, 0x3e, 0xe8, 0x43, - 0x52, 0x37, 0x46, 0x20, 0x29, 0x9f, 0xdf, 0x83, 0x52, 0xeb, 0xb1, 0x50, 0xea, 0xf6, 0x79, 0x50, - 0xca, 0x97, 0x13, 0xc1, 0x52, 0xf5, 0x38, 0x2c, 0x75, 0xeb, 0x1c, 0x2c, 0xe5, 0x8b, 0x0a, 0x83, - 0xa9, 0xf5, 0x58, 0x30, 0x75, 0xfb, 0x3c, 0x30, 0x15, 0x14, 0x2b, 0x8c, 0xa6, 0xbe, 0x15, 0x41, - 0x53, 0x4b, 0x43, 0xd1, 0x94, 0xcf, 0xcd, 0xe0, 0xd4, 0x87, 0xfd, 0x70, 0xea, 0xc6, 0x08, 0x38, - 0x15, 0x28, 0x96, 0xe3, 0xa9, 0x7a, 0x1c, 0x9e, 0xba, 0x75, 0x0e, 0x9e, 0x0a, 0x74, 0x11, 0x02, - 0x54, 0x5b, 0xf1, 0x80, 0xea, 0xce, 0xb9, 0x80, 0xca, 0x97, 0x16, 0x45, 0x54, 0xf5, 0x38, 0x44, - 0x75, 0xeb, 0x1c, 0x44, 0xd5, 0x57, 0x32, 0x06, 0xa9, 0xd4, 0x91, 0x90, 0xea, 0x9d, 0x31, 0x21, - 0x95, 0x2f, 0x3a, 0x0e, 0x53, 0xe9, 0xa3, 0x31, 0x55, 0x79, 0x5c, 0x4c, 0xe5, 0xbf, 0x24, 0x16, - 0x54, 0xa9, 0x23, 0x41, 0xd5, 0x3b, 0x63, 0x82, 0xaa, 0xbe, 0x8a, 0x44, 0x51, 0xd5, 0x56, 0x3c, - 0xaa, 0xba, 0x73, 0x2e, 0xaa, 0x0a, 0x5a, 0x31, 0x02, 0xab, 0x96, 0x43, 0xb0, 0xea, 0xad, 0x21, - 0xb0, 0xca, 0x67, 0x25, 0xb8, 0xea, 0xbb, 0x03, 0xb8, 0x4a, 0x1a, 0x85, 0xab, 0x7c, 0x5e, 0x1f, - 0x58, 0xd5, 0xe3, 0x80, 0xd5, 0xad, 0x73, 0x80, 0x55, 0xd0, 0x6f, 0x42, 0xc8, 0xea, 0xd9, 0x10, - 0x64, 0x75, 0xf7, 0x7c, 0x64, 0xe5, 0xcb, 0xeb, 0x83, 0x56, 0xea, 0x48, 0x68, 0xf5, 0xce, 0x98, - 0xd0, 0x2a, 0x68, 0xc1, 0x18, 0x6c, 0xf5, 0x5e, 0x14, 0x5b, 0x5d, 0x1f, 0x8e, 0xad, 0x7c, 0x31, - 0x1c, 0x5c, 0xad, 0xc7, 0x82, 0xab, 0xdb, 0xe7, 0x81, 0xab, 0x60, 0x36, 0x0b, 0xa3, 0xab, 0xad, - 0x78, 0x74, 0x75, 0xe7, 0x5c, 0x74, 0x15, 0x74, 0xa4, 0x08, 0xbc, 0x5a, 0x8f, 0x85, 0x57, 0xb7, - 0xcf, 0x83, 0x57, 0x7d, 0x53, 0x2d, 0xc7, 0x57, 0x2f, 0x86, 0xe2, 0xab, 0xfb, 0xe3, 0xe0, 0x2b, - 0x5f, 0xe8, 0x00, 0xc0, 0xfa, 0x64, 0x38, 0xc0, 0xfa, 0xa5, 0x0b, 0x5c, 0xd6, 0x19, 0x8b, 0xb0, - 0xbe, 0x3b, 0x80, 0xb0, 0xa4, 0x51, 0x08, 0x2b, 0x18, 0x19, 0x1e, 0xc4, 0xaa, 0xc5, 0x00, 0xa2, - 0xb7, 0x47, 0x03, 0xa2, 0x60, 0x21, 0x0f, 0x10, 0xd1, 0x07, 0x7d, 0x88, 0xe8, 0xc6, 0xb9, 0x7e, - 0x71, 0x21, 0x48, 0x54, 0x19, 0x84, 0x44, 0x37, 0x47, 0x42, 0x22, 0x5f, 0x42, 0x80, 0x89, 0xd6, - 0x63, 0x31, 0xd1, 0xed, 0xf3, 0x30, 0x51, 0xd0, 0x15, 0xc2, 0xa0, 0x68, 0x2b, 0x1e, 0x14, 0xdd, - 0x39, 0x17, 0x14, 0xf5, 0x2d, 0x5b, 0x1e, 0x2a, 0xaa, 0xc7, 0xa1, 0xa2, 0x5b, 0xe7, 0xa0, 0xa2, - 0xf0, 0xb2, 0xe5, 0xc3, 0xa2, 0xd6, 0x30, 0x58, 0x74, 0x6f, 0x0c, 0x58, 0x14, 0x18, 0x73, 0x7d, - 0xb8, 0xe8, 0xa3, 0x7e, 0x5c, 0x24, 0x8d, 0xc2, 0x45, 0x41, 0x27, 0xf2, 0x80, 0xd1, 0x56, 0x3c, - 0x30, 0xba, 0x73, 0x2e, 0x30, 0x0a, 0x8f, 0xeb, 0x10, 0x32, 0xfa, 0xa8, 0x1f, 0x19, 0x49, 0xa3, - 0x90, 0x51, 0x50, 0x1e, 0x0f, 0x1a, 0xd5, 0xe3, 0xa0, 0xd1, 0xad, 0x73, 0xa0, 0x51, 0x68, 0xba, - 0x0f, 0xb0, 0xd1, 0x5f, 0x1d, 0x1f, 0x1b, 0xbd, 0xf7, 0xba, 0x6e, 0x4b, 0xe7, 0x83, 0xa3, 0x8f, - 0xfa, 0xc1, 0x91, 0x34, 0x0a, 0x1c, 0x05, 0xfa, 0xf0, 0xd0, 0x51, 0xe7, 0x5c, 0x74, 0xf4, 0xe0, - 0x02, 0xe8, 0xc8, 0x97, 0xff, 0xa5, 0xe1, 0xd1, 0x9b, 0xe2, 0x5b, 0x11, 0x90, 0xf4, 0xcf, 0xb3, - 0x30, 0xc5, 0xbf, 0x53, 0x15, 0xb9, 0xdb, 0x49, 0x78, 0x9d, 0xbb, 0x9d, 0xd0, 0x77, 0xe0, 0x9a, - 0xff, 0x40, 0x3d, 0x4f, 0x14, 0x1e, 0xf1, 0xa2, 0xb5, 0x2d, 0xed, 0x90, 0xae, 0x3d, 0x59, 0x79, - 0xde, 0xcf, 0xf2, 0xd8, 0xb6, 0x3a, 0x2c, 0xf2, 0x85, 0x9e, 0xe7, 0xa3, 0x55, 0x32, 0x28, 0xa8, - 0x91, 0x75, 0xfe, 0x85, 0x82, 0x83, 0x57, 0xde, 0x71, 0xd6, 0xd7, 0x88, 0xd1, 0x46, 0xdf, 0x82, - 0x42, 0xcf, 0xc1, 0xb6, 0xd2, 0xb5, 0x0d, 0xcb, 0x36, 0x5c, 0x16, 0x6d, 0x22, 0x54, 0xc4, 0x2f, - 0x4e, 0x97, 0x66, 0x76, 0x1c, 0x6c, 0x6f, 0x73, 0xba, 0x3c, 0xd3, 0x0b, 0x3d, 0x79, 0x5f, 0xf6, - 0x9a, 0x1c, 0xff, 0xcb, 0x5e, 0xcf, 0x40, 0xa4, 0xbe, 0x04, 0xe1, 0x75, 0x89, 0x5d, 0xc3, 0x14, - 0xbf, 0x84, 0xaa, 0x7a, 0x68, 0xe9, 0xa1, 0xd7, 0x31, 0x95, 0xec, 0x28, 0x11, 0x3d, 0x81, 0xa2, - 0x6d, 0xf5, 0xe8, 0x6d, 0x2b, 0x5d, 0xab, 0x6d, 0x68, 0x27, 0xd4, 0x60, 0x28, 0xc6, 0x9f, 0x99, - 0xb2, 0x8c, 0xdb, 0x34, 0x9f, 0x5c, 0xb0, 0xc3, 0x8f, 0xa8, 0x09, 0xf4, 0xa6, 0x15, 0x4f, 0x0a, - 0x1a, 0xb8, 0xdb, 0x7a, 0xe4, 0x0d, 0xf3, 0x2f, 0x54, 0xc3, 0xe5, 0x72, 0xe1, 0x95, 0xff, 0x1b, - 0x3d, 0x86, 0x19, 0x7a, 0xeb, 0x2f, 0x69, 0x73, 0xab, 0xe7, 0x72, 0x7b, 0xe1, 0x8d, 0x32, 0xfb, - 0xfe, 0x5b, 0xd9, 0xfb, 0xfe, 0x5b, 0x79, 0x95, 0x7f, 0xff, 0x8d, 0x1d, 0xb0, 0x7e, 0xfe, 0x1f, - 0x97, 0x04, 0x39, 0xef, 0xf9, 0x78, 0x58, 0x3d, 0x17, 0x3d, 0x80, 0xcb, 0x1d, 0xf5, 0x98, 0x5e, - 0x1e, 0xac, 0x78, 0x06, 0x0d, 0xbd, 0x2e, 0x8d, 0x7d, 0x74, 0x0c, 0x75, 0xd4, 0x63, 0xfa, 0x31, - 0x34, 0x96, 0x44, 0xbf, 0x64, 0x72, 0x03, 0x66, 0x78, 0x18, 0x02, 0xfb, 0xd0, 0x51, 0x89, 0xe6, - 0xe4, 0x5f, 0xbd, 0x60, 0xdf, 0x3a, 0xba, 0x05, 0x45, 0xdd, 0x70, 0x5c, 0xc3, 0xd4, 0x5c, 0x7e, - 0x2f, 0x31, 0xbb, 0xd9, 0xb7, 0xe0, 0x51, 0xd9, 0xe5, 0xc3, 0x2d, 0x98, 0xd5, 0xda, 0x86, 0x6f, - 0x26, 0xb2, 0x85, 0x7b, 0x76, 0xe8, 0xb4, 0x50, 0xa5, 0x79, 0xfb, 0x8f, 0xfa, 0x4b, 0x5a, 0x94, - 0x8c, 0xaa, 0x50, 0xda, 0x57, 0x5d, 0xfc, 0x4a, 0x3d, 0x51, 0xbc, 0x28, 0xbb, 0x3c, 0x8d, 0x2c, - 0xbe, 0x76, 0x76, 0xba, 0x54, 0x78, 0xc2, 0x92, 0x06, 0x82, 0xed, 0x0a, 0xfb, 0xa1, 0x04, 0x1d, - 0xdd, 0x81, 0x92, 0xea, 0x9c, 0x98, 0x1a, 0xed, 0x51, 0xd8, 0x74, 0x7a, 0x0e, 0xb5, 0xf2, 0xb3, - 0x72, 0x91, 0x92, 0xab, 0x1e, 0x15, 0x7d, 0x00, 0x0b, 0xfc, 0xf3, 0x03, 0xaf, 0x54, 0x5b, 0x57, - 0x68, 0x2f, 0x0c, 0x86, 0xbb, 0x48, 0x79, 0xae, 0xb2, 0xcf, 0x0d, 0x90, 0x0c, 0xa4, 0xeb, 0x85, - 0xaf, 0xf5, 0x65, 0xd7, 0x16, 0x83, 0x98, 0x5f, 0xcb, 0x64, 0x67, 0xc4, 0xc2, 0x5a, 0x26, 0x5b, - 0x14, 0x4b, 0xd2, 0x0f, 0xd2, 0x50, 0x22, 0x13, 0xae, 0xe3, 0x18, 0x96, 0x59, 0xf7, 0x1d, 0x5e, - 0xfd, 0x61, 0x24, 0xd0, 0x80, 0x27, 0xff, 0x19, 0x2d, 0xd1, 0xc0, 0x32, 0x62, 0xd8, 0xfa, 0x1f, - 0x1d, 0x49, 0xcb, 0xc0, 0x48, 0x34, 0xc4, 0x67, 0x05, 0xa6, 0x1c, 0xab, 0x67, 0x6b, 0xde, 0x55, - 0xf8, 0xf7, 0x86, 0xcc, 0xf0, 0xa1, 0x17, 0x96, 0x9b, 0x94, 0x41, 0xe6, 0x8c, 0xe8, 0x13, 0x28, - 0xb1, 0x5f, 0x34, 0x98, 0x86, 0x06, 0xb3, 0xb0, 0x48, 0xa1, 0x07, 0x63, 0xcb, 0xda, 0xe0, 0x8c, - 0x72, 0xd1, 0x89, 0x3c, 0xa3, 0x8f, 0xe0, 0x4d, 0xd3, 0x52, 0x3a, 0xb8, 0x63, 0xb1, 0x99, 0x9d, - 0x8c, 0x05, 0x5d, 0x51, 0x5d, 0x85, 0x17, 0x9a, 0x79, 0x50, 0xce, 0x9b, 0xd6, 0x26, 0xcd, 0x22, - 0xf3, 0x1c, 0x2b, 0x2e, 0x93, 0x2b, 0x95, 0x61, 0x8a, 0xfd, 0x42, 0x39, 0x98, 0x7c, 0xda, 0xaa, - 0xd7, 0x64, 0x71, 0x02, 0xcd, 0x40, 0xf6, 0xb1, 0xfc, 0x74, 0x53, 0x69, 0x3e, 0xdb, 0x10, 0x05, - 0x94, 0x87, 0x69, 0xf9, 0xe9, 0xd3, 0x96, 0xb2, 0xfe, 0x5c, 0x4c, 0x49, 0x77, 0xa0, 0x18, 0x2d, - 0x11, 0x02, 0x98, 0x92, 0x6b, 0x9b, 0x4f, 0xe9, 0xfd, 0xef, 0x39, 0x98, 0xdc, 0x78, 0x5a, 0x5d, - 0xd9, 0x10, 0x05, 0xe9, 0x4f, 0x04, 0x98, 0xa9, 0xb0, 0x0f, 0x1a, 0x30, 0x0f, 0x8a, 0x0f, 0xfa, - 0x5c, 0x1b, 0xde, 0x88, 0x47, 0x8e, 0xf1, 0x2e, 0x0d, 0x2b, 0x90, 0xe5, 0x63, 0xcc, 0x0b, 0xba, - 0x58, 0x1a, 0x8e, 0x17, 0xe8, 0xd6, 0x9a, 0xe7, 0x38, 0xe7, 0xb1, 0xa1, 0x26, 0x88, 0xaa, 0xa7, - 0x5b, 0x85, 0x97, 0x64, 0xb8, 0xfb, 0x5c, 0x5f, 0x33, 0x78, 0x23, 0x46, 0x8d, 0x92, 0xdf, 0xcf, - 0x7c, 0xfe, 0xe3, 0xa5, 0x09, 0xe9, 0x4f, 0x33, 0x50, 0xa8, 0x84, 0x3f, 0xde, 0x80, 0x1a, 0x7d, - 0x95, 0x8d, 0xb3, 0x82, 0x22, 0x1c, 0xe5, 0x11, 0x9f, 0xc5, 0xc9, 0x05, 0x5f, 0x8a, 0x60, 0x75, - 0xbf, 0x3e, 0xc2, 0x2b, 0x24, 0x5c, 0xf9, 0x80, 0x71, 0xe1, 0xdf, 0xa5, 0xfd, 0x55, 0xb5, 0x0c, - 0x93, 0x2c, 0xb0, 0x4f, 0x18, 0xb8, 0x73, 0x80, 0xce, 0xa7, 0xc4, 0x4a, 0x27, 0xe9, 0x32, 0xcb, - 0x46, 0x56, 0xe1, 0xd6, 0x6b, 0xdd, 0xb0, 0x18, 0xd8, 0x2e, 0x17, 0xff, 0xe0, 0x64, 0x8f, 0xdd, - 0xb0, 0xf9, 0xff, 0xd1, 0x63, 0x8f, 0xbc, 0x0f, 0xfd, 0x1a, 0x94, 0x34, 0xab, 0xdd, 0x66, 0xc6, - 0x1d, 0x9b, 0x7f, 0x07, 0xef, 0xdc, 0xa1, 0x45, 0xe0, 0xdf, 0x18, 0x2d, 0xfb, 0xdf, 0x1a, 0x2d, - 0xcb, 0xfc, 0x5b, 0xa3, 0xa1, 0x70, 0x8a, 0xa2, 0x2f, 0x8c, 0x4d, 0xdb, 0x7d, 0x91, 0x1d, 0xd3, - 0xaf, 0x13, 0xd9, 0xc1, 0xe2, 0x61, 0x78, 0xcf, 0xfb, 0x43, 0x81, 0xfb, 0xd5, 0x6d, 0x58, 0xd6, - 0x61, 0xcf, 0xf7, 0x56, 0x5a, 0x08, 0xdf, 0x97, 0x19, 0x38, 0x9d, 0xd3, 0xa0, 0xa9, 0xb8, 0x05, - 0x3f, 0xf5, 0xe5, 0x16, 0xfc, 0x1b, 0x30, 0xd3, 0xb5, 0xf1, 0x1e, 0x76, 0xb5, 0x03, 0xc5, 0xec, - 0x75, 0x78, 0xc4, 0x58, 0xde, 0xa3, 0x6d, 0xf5, 0x3a, 0xe8, 0x1e, 0x88, 0x7e, 0x16, 0x0e, 0xb8, - 0xbd, 0xcb, 0xda, 0x3c, 0x3a, 0x87, 0xe7, 0xd2, 0xff, 0x14, 0x60, 0x2e, 0x52, 0x27, 0x3e, 0xa6, - 0xd6, 0x20, 0xaf, 0xfb, 0x26, 0x96, 0x33, 0x2f, 0x5c, 0x30, 0x1c, 0x21, 0xcc, 0x8c, 0x14, 0xb8, - 0xe2, 0xbd, 0x96, 0x7e, 0x5d, 0x21, 0x10, 0x9b, 0xba, 0xa0, 0xd8, 0xcb, 0x81, 0x9c, 0xd5, 0xd0, - 0x0b, 0xfc, 0x41, 0x96, 0x1e, 0x6b, 0x90, 0x49, 0xff, 0x4b, 0x00, 0x91, 0xbe, 0xe0, 0x31, 0xc6, - 0x7a, 0x22, 0x53, 0xa6, 0x17, 0xf7, 0x93, 0x1a, 0x3f, 0x26, 0x2c, 0xf2, 0x45, 0x98, 0x74, 0xdf, - 0x17, 0x61, 0xe2, 0xe6, 0xcf, 0xcc, 0x97, 0x9c, 0x3f, 0xa5, 0x1f, 0x0b, 0x50, 0xf4, 0xab, 0xcd, - 0x3e, 0x05, 0x39, 0xe2, 0xae, 0xd7, 0xd7, 0xfb, 0xdc, 0xa1, 0x77, 0x27, 0xcd, 0x58, 0x5f, 0xa7, - 0x0c, 0xdf, 0x49, 0xc3, 0x3e, 0xd3, 0xf7, 0xb7, 0xbc, 0xee, 0x48, 0x8a, 0x58, 0x0d, 0x2e, 0x03, - 0x79, 0x8d, 0x98, 0xbb, 0xaf, 0xc2, 0x31, 0xef, 0x71, 0x48, 0x81, 0xb4, 0x47, 0x11, 0x2d, 0x8d, - 0x35, 0xbf, 0x7b, 0x5a, 0x62, 0x1d, 0xf0, 0xa7, 0xe1, 0x96, 0x60, 0xc1, 0xe4, 0x8f, 0x20, 0x7d, - 0xa4, 0xb6, 0x47, 0x79, 0x22, 0x46, 0x5a, 0x4e, 0x26, 0xb9, 0xd1, 0xe3, 0xc8, 0x1d, 0x2a, 0xa9, - 0xe1, 0x9b, 0x71, 0x83, 0x2a, 0x8d, 0xdc, 0xb5, 0xf2, 0xed, 0xe8, 0x00, 0x1a, 0xf9, 0xfa, 0xf0, - 0x48, 0x7a, 0x3f, 0xf3, 0x93, 0x1f, 0x2f, 0x09, 0xd2, 0x87, 0x80, 0x88, 0xad, 0xe3, 0x3e, 0xeb, - 0x59, 0x76, 0x70, 0x1f, 0x4d, 0x7f, 0x7c, 0xcf, 0x64, 0x7c, 0x7c, 0x8f, 0x74, 0x19, 0xe6, 0x22, - 0xdc, 0x6c, 0x06, 0x92, 0xbe, 0x0d, 0x6f, 0x3c, 0xb1, 0x1c, 0xc7, 0xe8, 0x36, 0x7b, 0xbb, 0x6c, - 0xa8, 0x93, 0xf5, 0xca, 0x9f, 0x73, 0xb3, 0x5d, 0xba, 0xd3, 0x63, 0xb2, 0xb9, 0x29, 0x27, 0xfb, - 0xcf, 0xd2, 0xbf, 0x12, 0xe0, 0xea, 0x20, 0x27, 0xd3, 0x72, 0x5c, 0x88, 0xf0, 0xb4, 0x66, 0x05, - 0xd7, 0x25, 0x9e, 0xdf, 0x5b, 0xbd, 0xec, 0xc4, 0xf6, 0xe6, 0xef, 0x54, 0x3a, 0x2a, 0x9d, 0x93, - 0x78, 0xf4, 0x7e, 0x91, 0x93, 0x37, 0x19, 0x35, 0x98, 0x9e, 0x32, 0xe3, 0x4d, 0x4f, 0xff, 0x5b, - 0x80, 0xd9, 0x16, 0x36, 0x55, 0xd3, 0x25, 0xf3, 0x7e, 0xaf, 0xc3, 0xc2, 0xaa, 0x4b, 0x90, 0xb6, - 0x95, 0x1e, 0x2d, 0xba, 0x20, 0xa7, 0xec, 0x1d, 0x74, 0x13, 0x0a, 0x74, 0x6d, 0x09, 0xd9, 0x6a, - 0xc2, 0xdd, 0x8c, 0x4c, 0xbd, 0xd5, 0x65, 0xcf, 0x10, 0x7b, 0x0b, 0x80, 0x66, 0x62, 0x18, 0x28, - 0x4d, 0x73, 0xe4, 0x08, 0xc5, 0x47, 0x40, 0x34, 0xfa, 0x27, 0x10, 0xc2, 0x1c, 0xfc, 0x0b, 0x94, - 0xea, 0x4b, 0x59, 0x82, 0x3c, 0xcb, 0xc6, 0xc4, 0x4c, 0xd2, 0x3c, 0x40, 0x49, 0x4c, 0xce, 0x63, - 0xb8, 0xe4, 0xbc, 0x6c, 0x2b, 0x5d, 0x4b, 0x77, 0x14, 0xad, 0xdb, 0xe3, 0x9e, 0xd3, 0xec, 0xeb, - 0xb2, 0x42, 0xe5, 0xf2, 0xd9, 0xe9, 0xd2, 0x6c, 0xf3, 0xd9, 0xc6, 0xb6, 0xa5, 0x3b, 0xd5, 0xed, - 0x1d, 0xe6, 0x37, 0xed, 0xc8, 0xb3, 0xce, 0xcb, 0x36, 0x25, 0x75, 0x7b, 0x9c, 0x24, 0x7d, 0x96, - 0x02, 0x44, 0xaf, 0xe7, 0xa8, 0xd0, 0xfb, 0x2d, 0xbc, 0xe6, 0xb6, 0x60, 0x51, 0x0b, 0x54, 0xa1, - 0x38, 0x86, 0x49, 0xec, 0x7b, 0xd5, 0x71, 0xbd, 0x72, 0xf3, 0x71, 0x13, 0x7b, 0xab, 0x60, 0xbf, - 0x26, 0x79, 0x3b, 0x5e, 0x0b, 0x49, 0x6c, 0x12, 0x81, 0x1b, 0xaa, 0xe3, 0xbf, 0xf0, 0x1e, 0xe4, - 0x5c, 0xca, 0xe7, 0x5d, 0x7e, 0x92, 0xa9, 0xcc, 0x9c, 0x9d, 0x2e, 0x65, 0x99, 0xb0, 0xc6, 0xaa, - 0x9c, 0x65, 0xc9, 0x0d, 0x1d, 0x2d, 0x43, 0xde, 0x30, 0x1d, 0x57, 0x25, 0x45, 0xe2, 0x5b, 0x0e, - 0x05, 0x16, 0x68, 0xdb, 0xe0, 0xe4, 0xc6, 0xaa, 0x0c, 0x5e, 0x16, 0x1a, 0xcf, 0x56, 0xf4, 0x19, - 0xd8, 0xa6, 0x3c, 0x0d, 0xbe, 0x94, 0x0b, 0x1e, 0x95, 0x5d, 0xcf, 0xd5, 0x84, 0xb9, 0x88, 0x26, - 0xf8, 0xc2, 0xfc, 0x61, 0x74, 0xc2, 0x09, 0x5b, 0xa7, 0xde, 0xc7, 0xe9, 0xcb, 0x35, 0x53, 0xb3, - 0x74, 0x3e, 0x52, 0xa3, 0x13, 0x4f, 0x0b, 0x4a, 0x6b, 0x96, 0x61, 0x12, 0xf4, 0xe8, 0x55, 0x75, - 0x05, 0x8a, 0xbb, 0x86, 0xa9, 0xda, 0x27, 0x8a, 0xe7, 0x28, 0x2e, 0x9c, 0xe7, 0x28, 0x2e, 0x17, - 0x18, 0x07, 0x7f, 0x94, 0x7e, 0x2e, 0x80, 0x18, 0x88, 0xe5, 0x05, 0xfd, 0x06, 0x80, 0xd6, 0xee, - 0x39, 0x2e, 0xb6, 0xbd, 0x09, 0x60, 0x86, 0x05, 0xa4, 0x55, 0x19, 0xb5, 0xb1, 0x2a, 0xe7, 0x78, - 0x86, 0x86, 0x8e, 0x6e, 0x46, 0xef, 0x9a, 0x99, 0xac, 0xc0, 0xd9, 0xc0, 0x0d, 0x33, 0x64, 0x46, - 0x71, 0x5c, 0xcb, 0xf6, 0xf5, 0xcc, 0x67, 0x14, 0xef, 0x16, 0x2e, 0x7a, 0xdb, 0x04, 0xa6, 0x31, - 0xa7, 0x45, 0x62, 0xde, 0x1e, 0x61, 0xbf, 0x4a, 0x99, 0xf3, 0xab, 0xc4, 0x38, 0xbc, 0x2a, 0xfd, - 0xbe, 0x00, 0xa5, 0x2a, 0x1b, 0xe8, 0xfe, 0xe4, 0x31, 0x62, 0xb1, 0x5c, 0x85, 0xac, 0x7b, 0x6c, - 0x2a, 0x1d, 0xec, 0x7f, 0x38, 0xea, 0x02, 0xd7, 0x64, 0x4e, 0xbb, 0xec, 0x91, 0x7e, 0x8b, 0x94, - 0x6f, 0x84, 0xf0, 0x99, 0x78, 0xac, 0x9d, 0x12, 0x9f, 0x89, 0xd9, 0xa9, 0xf7, 0x9b, 0x64, 0x42, - 0x1d, 0xb0, 0x24, 0x51, 0x11, 0x20, 0xf4, 0x45, 0x30, 0xfe, 0xed, 0xf5, 0x95, 0x55, 0x65, 0x67, - 0xab, 0xfa, 0x74, 0x73, 0xb3, 0xd1, 0x6a, 0xd5, 0x56, 0x45, 0x01, 0x89, 0x30, 0x13, 0xf9, 0x9e, - 0x58, 0x8a, 0x7d, 0x8d, 0xfd, 0xfe, 0x3b, 0x50, 0x88, 0x6c, 0x1f, 0xa1, 0x12, 0xe4, 0x37, 0x6a, - 0x2b, 0xcd, 0x5a, 0xfd, 0xe9, 0xc6, 0x2a, 0x05, 0xb2, 0x79, 0x98, 0xde, 0xaa, 0xad, 0xc8, 0xb5, - 0x66, 0x4b, 0x14, 0xee, 0xff, 0x05, 0x80, 0xe0, 0x4b, 0x86, 0xe4, 0xd5, 0xeb, 0xb5, 0x8f, 0x95, - 0xe7, 0x2b, 0x1b, 0x3b, 0xb5, 0xa6, 0x38, 0x81, 0x10, 0x14, 0x2b, 0x2b, 0xad, 0x6a, 0x5d, 0x91, - 0x6b, 0xcd, 0xed, 0xa7, 0x5b, 0xcd, 0x9a, 0xf7, 0xd1, 0xf7, 0xfb, 0xab, 0x30, 0x13, 0xbe, 0x2b, - 0x0c, 0xcd, 0x41, 0xa9, 0x5a, 0xaf, 0x55, 0xd7, 0x95, 0xe7, 0x8d, 0x15, 0xe5, 0xd9, 0x4e, 0x6d, - 0x87, 0x20, 0x5f, 0x52, 0x13, 0x4a, 0x7c, 0xbc, 0xb3, 0x41, 0x40, 0x73, 0x09, 0xf2, 0xec, 0x99, - 0x7e, 0xaa, 0x4c, 0x4c, 0xdd, 0xdf, 0x84, 0x7c, 0xe8, 0x0e, 0x73, 0xf2, 0xba, 0xed, 0x9d, 0x66, - 0x5d, 0x69, 0x35, 0x36, 0x6b, 0xcd, 0xd6, 0xca, 0xe6, 0x36, 0x93, 0x41, 0x69, 0x2b, 0x95, 0xa7, - 0x72, 0x4b, 0x14, 0xfc, 0xe7, 0xd6, 0xd3, 0x9d, 0x6a, 0xdd, 0xab, 0xb5, 0x94, 0xc9, 0xa6, 0xc5, - 0xf4, 0xfd, 0x63, 0xb8, 0x3a, 0xe4, 0xda, 0x2c, 0x52, 0xe9, 0x1d, 0x93, 0xde, 0xe7, 0x2c, 0x4e, - 0xa0, 0x02, 0xe4, 0x48, 0x4f, 0xa5, 0x41, 0xf5, 0xa2, 0x80, 0xb2, 0x90, 0x39, 0x70, 0xdd, 0xae, - 0x98, 0x42, 0x53, 0x90, 0x72, 0x1e, 0x89, 0x69, 0xf2, 0x7f, 0xdf, 0x11, 0x33, 0x04, 0xba, 0xab, - 0x9f, 0xf6, 0x6c, 0x2c, 0x4e, 0x12, 0xf8, 0xdf, 0x73, 0xb0, 0xbd, 0x67, 0xb4, 0xb1, 0x38, 0x4d, - 0x58, 0xcc, 0x5e, 0xbb, 0x2d, 0x66, 0xa5, 0x4c, 0x76, 0x4a, 0x9c, 0xba, 0x7f, 0x03, 0x42, 0xb7, - 0x97, 0x10, 0xf4, 0xbf, 0xa1, 0xba, 0xd8, 0x71, 0xc5, 0x09, 0x34, 0x0d, 0xe9, 0x95, 0x76, 0x5b, - 0x14, 0x1e, 0x7e, 0x3e, 0x09, 0x59, 0xef, 0x4b, 0x5c, 0x68, 0x03, 0x26, 0x29, 0xd2, 0x45, 0x4b, - 0xc3, 0x31, 0x30, 0x1d, 0xf6, 0x0b, 0xd7, 0xcf, 0x03, 0xc9, 0xd2, 0x04, 0xfa, 0x8b, 0x90, 0x0f, - 0x61, 0x03, 0x34, 0x74, 0x3f, 0x3f, 0x82, 0x87, 0x16, 0x6e, 0x9f, 0x97, 0xcd, 0x97, 0xff, 0x02, - 0x72, 0xbe, 0x59, 0x81, 0x6e, 0x8e, 0x32, 0x3a, 0x3c, 0xd9, 0xa3, 0x2d, 0x13, 0x32, 0x4a, 0xa5, - 0x89, 0x77, 0x05, 0x64, 0x03, 0x1a, 0xb4, 0x00, 0x50, 0x9c, 0xcf, 0xec, 0x50, 0x13, 0x63, 0xe1, - 0xfe, 0x58, 0xb9, 0x83, 0x77, 0x12, 0x65, 0x05, 0x66, 0x4c, 0xbc, 0xb2, 0x06, 0x8c, 0xa4, 0x78, - 0x65, 0xc5, 0x58, 0x43, 0xb4, 0x31, 0x42, 0xeb, 0x41, 0xac, 0xfc, 0xc1, 0x95, 0x33, 0x56, 0x7e, - 0xcc, 0xb2, 0x22, 0x4d, 0xa0, 0x67, 0x90, 0x21, 0x73, 0x38, 0x8a, 0x03, 0x18, 0x7d, 0x6b, 0xc6, - 0xc2, 0xcd, 0x91, 0x79, 0x3c, 0x91, 0x95, 0x7b, 0x3f, 0xf9, 0x4f, 0x8b, 0x13, 0x3f, 0x39, 0x5b, - 0x14, 0x7e, 0x7e, 0xb6, 0x28, 0xfc, 0xd1, 0xd9, 0xa2, 0xf0, 0xc7, 0x67, 0x8b, 0xc2, 0x0f, 0x7f, - 0xb1, 0x38, 0xf1, 0xf3, 0x5f, 0x2c, 0x4e, 0xfc, 0xd1, 0x2f, 0x16, 0x27, 0x3e, 0x99, 0xe6, 0xdc, - 0xbb, 0x53, 0x74, 0x82, 0x7b, 0xf4, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x8a, 0x1d, 0x6f, 0x74, - 0x88, 0x85, 0x00, 0x00, + 0x73, 0x54, 0xf4, 0xae, 0xe9, 0x24, 0xac, 0x2f, 0xe9, 0x97, 0xe1, 0x52, 0x54, 0x74, 0x92, 0xe5, + 0x7e, 0xc1, 0x5b, 0x79, 0x0b, 0xdb, 0x89, 0x22, 0x54, 0x5f, 0xd7, 0x5c, 0x70, 0x92, 0x65, 0xfe, + 0x35, 0x01, 0xde, 0xa0, 0xb2, 0xe9, 0xd1, 0xcb, 0x3e, 0xb6, 0x37, 0xb1, 0xea, 0x24, 0x0a, 0xaf, + 0x6f, 0xc2, 0x14, 0x83, 0xc9, 0xb4, 0x7f, 0x4e, 0xae, 0xe6, 0x89, 0x99, 0xd1, 0x70, 0x2d, 0x9b, + 0x98, 0x19, 0x3c, 0x49, 0x52, 0x61, 0x21, 0xae, 0x14, 0x49, 0xd6, 0xf4, 0x37, 0x05, 0x98, 0xe5, + 0x16, 0x1e, 0xe9, 0xca, 0x95, 0x43, 0x62, 0xe0, 0xa0, 0x2a, 0xe4, 0x35, 0xfa, 0x4b, 0x71, 0x7b, + 0x1d, 0x4c, 0xe5, 0x17, 0x47, 0x19, 0x87, 0x8c, 0xad, 0xd9, 0xeb, 0x60, 0x62, 0x61, 0x7a, 0xbf, + 0x89, 0xa2, 0x42, 0x95, 0x1c, 0x69, 0x5e, 0xd2, 0x71, 0x44, 0xf3, 0x7a, 0x76, 0x1a, 0xd7, 0xc1, + 0x3f, 0x4e, 0x73, 0x25, 0xb0, 0x77, 0xf0, 0xec, 0x89, 0x1a, 0x14, 0x9f, 0xc0, 0x95, 0xd0, 0xd6, + 0x79, 0xb8, 0xe2, 0xa9, 0x0b, 0x54, 0x3c, 0xb4, 0xfd, 0x1e, 0x50, 0xd1, 0xc7, 0x10, 0xda, 0x60, + 0x57, 0x58, 0x9d, 0x3c, 0xa8, 0x72, 0x11, 0x75, 0xcc, 0x06, 0x52, 0x18, 0xdd, 0x41, 0x15, 0xc8, + 0xe2, 0x93, 0x8e, 0xa2, 0x63, 0x47, 0xe3, 0x13, 0x97, 0x14, 0x27, 0x90, 0x14, 0x65, 0xc0, 0x78, + 0x9f, 0xc6, 0x27, 0x1d, 0x42, 0x44, 0xbb, 0x64, 0xdd, 0xf4, 0xd6, 0x75, 0x5a, 0x6c, 0xe7, 0x7c, + 0x2c, 0x10, 0xf4, 0x14, 0x2e, 0xae, 0xe4, 0x2f, 0xe9, 0x4c, 0x84, 0xf4, 0x53, 0x01, 0xae, 0xc5, + 0xb6, 0x5a, 0x92, 0x0b, 0xd9, 0x87, 0x90, 0xa1, 0x95, 0x4f, 0x5d, 0xb0, 0xf2, 0x94, 0x4b, 0xfa, + 0x2c, 0xc5, 0xc7, 0xb8, 0x8c, 0x5b, 0x16, 0x51, 0x6c, 0xe2, 0x5b, 0x68, 0x4f, 0xa1, 0x70, 0x6c, + 0xb9, 0xd8, 0xf6, 0x9b, 0x3d, 0x75, 0xe1, 0x66, 0x9f, 0xa1, 0x02, 0xbc, 0x16, 0x7f, 0x0e, 0xb3, + 0xa6, 0x65, 0x2a, 0x51, 0xa1, 0x17, 0xef, 0x4b, 0x25, 0xd3, 0x32, 0x9f, 0x87, 0xe4, 0xfa, 0xf3, + 0x4c, 0x9f, 0x26, 0x92, 0x9c, 0x67, 0x7e, 0x2c, 0xc0, 0x9c, 0x6f, 0xe9, 0x24, 0x6c, 0xee, 0x7e, + 0x07, 0xd2, 0xa6, 0xf5, 0xea, 0x22, 0x5b, 0x94, 0x24, 0x3f, 0x59, 0xf5, 0xa2, 0x25, 0x4a, 0xb2, + 0xbe, 0xff, 0x2a, 0x05, 0xb9, 0x27, 0x95, 0x24, 0x6b, 0xf9, 0x21, 0xdf, 0xfe, 0x66, 0xed, 0x1d, + 0xd7, 0xdb, 0xfd, 0xf7, 0x95, 0x9f, 0x54, 0x36, 0x70, 0xcf, 0xeb, 0xed, 0x84, 0x0b, 0xad, 0x40, + 0x2e, 0xba, 0x51, 0x3a, 0xa6, 0xa6, 0x02, 0xae, 0x05, 0x0c, 0x93, 0x54, 0xae, 0xe7, 0x6a, 0x21, + 0xc4, 0xb8, 0x5a, 0x90, 0xd7, 0xf8, 0x96, 0x62, 0xea, 0x22, 0xaf, 0x09, 0x99, 0x88, 0x93, 0xe2, + 0x94, 0xf4, 0x0c, 0x80, 0x54, 0x27, 0xc9, 0x26, 0xf9, 0x51, 0x1a, 0x8a, 0x3b, 0x5d, 0xe7, 0x30, + 0xe1, 0xde, 0x57, 0x01, 0xe8, 0x74, 0x9d, 0x43, 0x32, 0x22, 0x4f, 0x4c, 0x5e, 0xe7, 0x73, 0xbc, + 0x38, 0xbc, 0x4a, 0x33, 0xbe, 0xe6, 0x89, 0x89, 0x6a, 0x5c, 0x08, 0x56, 0x02, 0x57, 0x90, 0x9b, + 0xa3, 0x90, 0x65, 0xf3, 0xc4, 0xdc, 0xc2, 0x3e, 0xa4, 0x64, 0x92, 0x30, 0x91, 0xf4, 0x21, 0x4c, + 0x93, 0x07, 0xc5, 0xb5, 0x2e, 0xd2, 0xcc, 0x53, 0x84, 0xa7, 0x69, 0xa1, 0x0f, 0x20, 0xc7, 0xb8, + 0xc9, 0xea, 0x37, 0x45, 0x57, 0xbf, 0xb8, 0xba, 0x70, 0x35, 0xd2, 0x75, 0x2f, 0x4b, 0x59, 0xc9, + 0x5a, 0x77, 0x09, 0x26, 0xf7, 0x2d, 0x5b, 0xf3, 0x0e, 0x73, 0xd9, 0x03, 0x6b, 0x4f, 0x06, 0x69, + 0xd6, 0x33, 0xd9, 0x9c, 0x08, 0xd2, 0x6f, 0x08, 0x50, 0xf2, 0x1b, 0x22, 0xc9, 0x05, 0xa1, 0x12, + 0xd1, 0xe2, 0xc5, 0x9b, 0x82, 0x28, 0x50, 0xfa, 0x37, 0xd4, 0x22, 0xd2, 0xac, 0x63, 0xda, 0x32, + 0x49, 0xf6, 0x94, 0x0f, 0x98, 0xa3, 0x4f, 0xea, 0xa2, 0xad, 0x4b, 0x7d, 0x7e, 0x1e, 0xc0, 0x25, + 0xa3, 0x4d, 0xe6, 0x73, 0xc3, 0x6d, 0xf5, 0x38, 0x6c, 0x73, 0xb1, 0x77, 0x6a, 0x3c, 0x17, 0xa4, + 0x55, 0xbc, 0x24, 0xe9, 0x77, 0xe9, 0x6e, 0x75, 0x50, 0x93, 0x24, 0x55, 0x5d, 0x87, 0x82, 0xcd, + 0x44, 0x13, 0xb3, 0xe6, 0x82, 0xda, 0x9e, 0xf1, 0x59, 0x89, 0xc2, 0x7f, 0x2b, 0x05, 0xa5, 0x67, + 0x5d, 0x6c, 0xf7, 0xbe, 0x49, 0xea, 0xbe, 0x0d, 0xa5, 0x57, 0xaa, 0xe1, 0x2a, 0xfb, 0x96, 0xad, + 0x74, 0x3b, 0xba, 0xea, 0x7a, 0xde, 0x26, 0x05, 0x42, 0x7e, 0x6c, 0xd9, 0xbb, 0x94, 0x88, 0x30, + 0xa0, 0x23, 0xd3, 0x7a, 0x65, 0x2a, 0x84, 0x4c, 0x81, 0xf2, 0x89, 0xc9, 0xb7, 0x90, 0x57, 0xbf, + 0xfb, 0xef, 0x4f, 0x97, 0x1e, 0x8d, 0xe5, 0x43, 0x46, 0xfd, 0xe5, 0xba, 0x5d, 0x43, 0x2f, 0xef, + 0xee, 0xd6, 0xd7, 0x64, 0x91, 0x8a, 0x7c, 0xc1, 0x24, 0x36, 0x4f, 0x4c, 0x47, 0xfa, 0xbb, 0x29, + 0x10, 0x03, 0x1d, 0x25, 0xd9, 0x90, 0x55, 0xc8, 0xbf, 0xec, 0x62, 0xdb, 0x78, 0x8d, 0x66, 0x04, + 0xce, 0x48, 0xa6, 0x9d, 0xfb, 0x30, 0xeb, 0x9e, 0x98, 0x0a, 0xf3, 0xf0, 0x63, 0x8e, 0x1f, 0x9e, + 0xc3, 0x42, 0xc9, 0x25, 0x65, 0x26, 0x74, 0xea, 0xf4, 0xe1, 0xa0, 0x4f, 0x60, 0x26, 0xa2, 0xad, + 0xf4, 0x57, 0xd3, 0x56, 0xfe, 0x55, 0x48, 0x51, 0x7f, 0x20, 0x00, 0xa2, 0x8a, 0xaa, 0xb3, 0x3d, + 0xfe, 0x6f, 0x4a, 0x7f, 0xba, 0x0b, 0x22, 0xf5, 0xc7, 0x54, 0x8c, 0x7d, 0xa5, 0x6d, 0x38, 0x8e, + 0x61, 0x1e, 0xf0, 0x0e, 0x55, 0xa4, 0xf4, 0xfa, 0xfe, 0x16, 0xa3, 0x4a, 0x7f, 0x05, 0xe6, 0x22, + 0x15, 0x48, 0xb2, 0xb1, 0x6f, 0xc0, 0xcc, 0x3e, 0x3b, 0x82, 0xa5, 0xc2, 0xf9, 0xf6, 0x60, 0x9e, + 0xd2, 0xd8, 0xfb, 0xa4, 0xff, 0x9a, 0x82, 0x4b, 0x32, 0x76, 0xac, 0xd6, 0x31, 0x4e, 0x5e, 0x85, + 0x35, 0xe0, 0x67, 0x2f, 0xca, 0x6b, 0x69, 0x32, 0xc7, 0x98, 0xd9, 0x32, 0x17, 0xdd, 0x63, 0x7f, + 0x7b, 0x74, 0x8f, 0x1d, 0xdc, 0x55, 0xe7, 0x3b, 0x75, 0x99, 0xc8, 0x4e, 0x9d, 0x05, 0x25, 0x76, + 0x7a, 0xac, 0x2b, 0x0e, 0x7e, 0x69, 0x76, 0xdb, 0x1e, 0x18, 0x2a, 0x8f, 0x2a, 0x64, 0x9d, 0xb1, + 0x34, 0xf0, 0xcb, 0xed, 0x6e, 0x9b, 0xda, 0xce, 0xab, 0x57, 0x48, 0x79, 0xcf, 0x4e, 0x97, 0x8a, + 0x91, 0x34, 0x47, 0x2e, 0x1a, 0xfe, 0x33, 0x91, 0x2e, 0xfd, 0x00, 0x2e, 0xf7, 0x29, 0x3b, 0x49, + 0x8b, 0xe7, 0x9f, 0xa7, 0xe1, 0x8d, 0xa8, 0xf8, 0xa4, 0x21, 0xce, 0x37, 0xbd, 0x41, 0x6b, 0x50, + 0x68, 0x1b, 0xe6, 0xeb, 0xed, 0x5e, 0xce, 0xb4, 0x0d, 0xd3, 0xa7, 0xc5, 0x75, 0x8d, 0xa9, 0xaf, + 0xb5, 0x6b, 0xa8, 0xb0, 0x10, 0xd7, 0x76, 0x49, 0xf6, 0x8f, 0xcf, 0x04, 0x98, 0x49, 0x7a, 0x5b, + 0xee, 0xf5, 0xbc, 0xe0, 0xa4, 0x26, 0x14, 0xbe, 0x86, 0x7d, 0xbc, 0xdf, 0x12, 0x00, 0x35, 0xed, + 0xae, 0x49, 0x40, 0xed, 0xa6, 0x75, 0x90, 0x64, 0x35, 0x2f, 0xc1, 0xa4, 0x61, 0xea, 0xf8, 0x84, + 0x56, 0x33, 0x23, 0xb3, 0x87, 0xc8, 0x51, 0x62, 0x7a, 0xac, 0xa3, 0x44, 0xe9, 0x13, 0x98, 0x8b, + 0x14, 0x31, 0xc9, 0xfa, 0xff, 0x59, 0x0a, 0xe6, 0x78, 0x45, 0x12, 0xdf, 0xc1, 0xfc, 0x36, 0x4c, + 0xb6, 0x88, 0xcc, 0x11, 0xed, 0x4c, 0xdf, 0xe9, 0xb5, 0x33, 0xcd, 0x8c, 0xbe, 0x07, 0xd0, 0xb1, + 0xf1, 0xb1, 0xc2, 0x58, 0xd3, 0x63, 0xb1, 0xe6, 0x08, 0x07, 0x25, 0xa0, 0xcf, 0x05, 0x28, 0x91, + 0x01, 0xdd, 0xb1, 0xad, 0x8e, 0xe5, 0x10, 0x9b, 0xc5, 0x19, 0x0f, 0xe6, 0x3c, 0x3b, 0x3b, 0x5d, + 0x2a, 0x6c, 0x19, 0xe6, 0x0e, 0x67, 0x6c, 0x36, 0xc6, 0x76, 0xf0, 0xf7, 0xc2, 0x1c, 0xca, 0x95, + 0x96, 0xa5, 0x1d, 0x05, 0x87, 0x63, 0x64, 0x66, 0xf1, 0xc5, 0x39, 0xd2, 0x1f, 0x09, 0x70, 0xe9, + 0x6b, 0xdb, 0x2e, 0xfe, 0xff, 0xa1, 0x6c, 0xe9, 0x39, 0x88, 0xf4, 0x47, 0xdd, 0xdc, 0xb7, 0x92, + 0xdc, 0xb8, 0xff, 0xdf, 0x02, 0xcc, 0x86, 0x04, 0x27, 0x69, 0xe0, 0xbc, 0xae, 0x9e, 0x0a, 0xcc, + 0x1d, 0xc6, 0x1d, 0x4f, 0x55, 0xf2, 0x0c, 0xcf, 0xce, 0x3a, 0x65, 0x19, 0x66, 0x30, 0x99, 0xc5, + 0xe8, 0x16, 0xef, 0x1e, 0x0b, 0x32, 0xe9, 0xdb, 0xd1, 0xcf, 0xfb, 0x19, 0x56, 0x7b, 0xd2, 0x2f, + 0x13, 0x0b, 0x2b, 0x3c, 0x28, 0x93, 0x1c, 0xf2, 0xff, 0x2c, 0x05, 0x57, 0x2a, 0xec, 0x08, 0xdc, + 0xf3, 0x09, 0x49, 0xb2, 0x23, 0xce, 0xc3, 0xf4, 0x31, 0xb6, 0x1d, 0xc3, 0x62, 0xab, 0x7d, 0x41, + 0xf6, 0x1e, 0xd1, 0x02, 0x64, 0x1d, 0x53, 0xed, 0x38, 0x87, 0x96, 0x77, 0x9c, 0xe8, 0x3f, 0xfb, + 0xfe, 0x2b, 0x93, 0xaf, 0xef, 0xbf, 0x32, 0x35, 0xda, 0x7f, 0x65, 0xfa, 0x2b, 0xf8, 0xaf, 0xf0, + 0xb3, 0xbb, 0x7f, 0x2b, 0xc0, 0xd5, 0x01, 0xcd, 0x25, 0xd9, 0x39, 0x7f, 0x08, 0x79, 0x8d, 0x0b, + 0x26, 0xeb, 0x03, 0x3b, 0x98, 0xac, 0x93, 0x6c, 0xaf, 0x09, 0x7d, 0xce, 0x4e, 0x97, 0xc0, 0x2b, + 0x6a, 0x7d, 0x8d, 0x2b, 0x87, 0xfc, 0xd6, 0xa5, 0xff, 0x06, 0x50, 0xaa, 0x9e, 0xb0, 0x4d, 0xf9, + 0x06, 0xb3, 0x4a, 0xd0, 0x63, 0xc8, 0x76, 0x6c, 0xeb, 0xd8, 0xf0, 0xaa, 0x51, 0x8c, 0x38, 0x2f, + 0x78, 0xd5, 0xe8, 0xe3, 0xda, 0xe1, 0x1c, 0xb2, 0xcf, 0x8b, 0x9a, 0x90, 0xdb, 0xb4, 0x34, 0xb5, + 0xf5, 0xd8, 0x68, 0x79, 0x03, 0xed, 0xdd, 0xf3, 0x05, 0x95, 0x7d, 0x9e, 0x1d, 0xd5, 0x3d, 0xf4, + 0x1a, 0xc1, 0x27, 0xa2, 0x3a, 0x64, 0x6b, 0xae, 0xdb, 0x21, 0x89, 0x7c, 0xfc, 0xdd, 0x19, 0x43, + 0x28, 0x61, 0xf1, 0x3c, 0x6e, 0x3d, 0x76, 0xd4, 0x84, 0xd9, 0x27, 0x34, 0x7e, 0xac, 0xd2, 0xb2, + 0xba, 0x7a, 0xc5, 0x32, 0xf7, 0x8d, 0x03, 0xbe, 0x4c, 0xdc, 0x1e, 0x43, 0xe6, 0x93, 0x4a, 0x43, + 0x1e, 0x14, 0x80, 0x56, 0x20, 0xdb, 0x78, 0xc4, 0x85, 0x31, 0x33, 0xf2, 0xd6, 0x18, 0xc2, 0x1a, + 0x8f, 0x64, 0x9f, 0x0d, 0xad, 0x43, 0x7e, 0xe5, 0xd3, 0xae, 0x8d, 0xb9, 0x94, 0xa9, 0xa1, 0x9e, + 0x13, 0xfd, 0x52, 0x28, 0x97, 0x1c, 0x66, 0x46, 0x3f, 0x80, 0x12, 0xd1, 0x5b, 0x53, 0xdd, 0x6b, + 0x79, 0xf2, 0xb2, 0x54, 0xde, 0xb7, 0xc6, 0x90, 0xe7, 0x73, 0x7a, 0x47, 0x02, 0x7d, 0xa2, 0x16, + 0x64, 0x28, 0x44, 0xda, 0x0b, 0x21, 0xc8, 0x74, 0x48, 0xd3, 0x08, 0xd4, 0x0d, 0x89, 0xfe, 0x46, + 0xef, 0xc0, 0xb4, 0x69, 0xe9, 0xd8, 0xeb, 0xcc, 0x85, 0xd5, 0x4b, 0x67, 0xa7, 0x4b, 0x53, 0xdb, + 0x96, 0xce, 0x6c, 0x1d, 0xfe, 0x4b, 0x9e, 0x22, 0x99, 0xea, 0xfa, 0xc2, 0x75, 0xc8, 0x90, 0x26, + 0x22, 0x73, 0xc8, 0x9e, 0xea, 0xe0, 0x5d, 0xdb, 0xe0, 0xd2, 0xbc, 0xc7, 0x85, 0x7f, 0x90, 0x82, + 0x54, 0xe3, 0x11, 0xb1, 0xe6, 0xf7, 0xba, 0xda, 0x11, 0x76, 0x79, 0x3a, 0x7f, 0xa2, 0x56, 0xbe, + 0x8d, 0xf7, 0x0d, 0x66, 0x74, 0xe5, 0x64, 0xfe, 0x84, 0xde, 0x02, 0x50, 0x35, 0x0d, 0x3b, 0x8e, + 0xe2, 0x85, 0x00, 0xe6, 0xe4, 0x1c, 0xa3, 0x6c, 0xe0, 0x1e, 0x61, 0x73, 0xb0, 0x66, 0x63, 0xd7, + 0xf3, 0xa1, 0x62, 0x4f, 0x84, 0xcd, 0xc5, 0xed, 0x8e, 0xe2, 0x5a, 0x47, 0xd8, 0xa4, 0x4d, 0x9a, + 0x23, 0xb3, 0x42, 0xbb, 0xd3, 0x24, 0x04, 0x32, 0xa1, 0x61, 0x53, 0x0f, 0x66, 0x9f, 0x9c, 0xec, + 0x3f, 0x13, 0x91, 0x36, 0x3e, 0x30, 0x78, 0x00, 0x5d, 0x4e, 0xe6, 0x4f, 0x44, 0x4b, 0x6a, 0xd7, + 0x3d, 0xa4, 0x2d, 0x91, 0x93, 0xe9, 0x6f, 0x74, 0x1b, 0x4a, 0xcc, 0xed, 0x52, 0xc1, 0xa6, 0xa6, + 0xd0, 0x79, 0x30, 0x47, 0x93, 0x0b, 0x8c, 0x5c, 0x35, 0x35, 0x32, 0xeb, 0xa1, 0x47, 0xc0, 0x09, + 0xca, 0x51, 0xdb, 0x21, 0x3a, 0x05, 0x92, 0x6b, 0xb5, 0x74, 0x76, 0xba, 0x94, 0x6f, 0xd0, 0x84, + 0x8d, 0xad, 0x06, 0x59, 0x4b, 0x58, 0xae, 0x8d, 0xb6, 0x53, 0xd7, 0x17, 0xfe, 0xb6, 0x00, 0xe9, + 0x27, 0x95, 0xc6, 0x85, 0x55, 0xe6, 0x15, 0x34, 0x1d, 0x2a, 0xe8, 0x1d, 0x28, 0xed, 0x19, 0xad, + 0x96, 0x61, 0x1e, 0x10, 0xfb, 0xea, 0x87, 0x58, 0xf3, 0x14, 0x56, 0xe4, 0xe4, 0x1d, 0x46, 0x45, + 0xd7, 0x21, 0xaf, 0xd9, 0x58, 0xc7, 0xa6, 0x6b, 0xa8, 0x2d, 0x87, 0x6b, 0x2e, 0x4c, 0x5a, 0xf8, + 0xab, 0x02, 0x4c, 0xd2, 0xce, 0x8a, 0xde, 0x84, 0x9c, 0x66, 0x99, 0xae, 0x6a, 0x98, 0x7c, 0xd6, + 0xc9, 0xc9, 0x01, 0x61, 0x68, 0xf1, 0x6e, 0xc0, 0x8c, 0xaa, 0x69, 0x56, 0xd7, 0x74, 0x15, 0x53, + 0x6d, 0x63, 0x5e, 0xcc, 0x3c, 0xa7, 0x6d, 0xab, 0x6d, 0x8c, 0x96, 0xc0, 0x7b, 0xf4, 0x23, 0x3b, + 0x73, 0x32, 0x70, 0xd2, 0x06, 0xee, 0x2d, 0x60, 0xc8, 0xf9, 0xbd, 0x9a, 0xd4, 0xb7, 0xeb, 0xf8, + 0x25, 0xa0, 0xbf, 0xd1, 0xbb, 0x70, 0xe9, 0x65, 0x57, 0x6d, 0x19, 0xfb, 0x74, 0xf3, 0x8b, 0x7a, + 0xa9, 0xd3, 0x97, 0xb1, 0xa2, 0x20, 0x3f, 0x8d, 0x4a, 0xa0, 0xef, 0xf4, 0x06, 0x41, 0x3a, 0x18, + 0x04, 0xcc, 0x65, 0x47, 0xea, 0xc1, 0xac, 0x8c, 0x5d, 0xbb, 0xd7, 0x64, 0xc1, 0xae, 0xd5, 0x63, + 0x6c, 0xba, 0xa4, 0xee, 0x56, 0x07, 0x33, 0x27, 0x11, 0xaf, 0xee, 0x3e, 0x01, 0xdd, 0x82, 0xa2, + 0xea, 0x92, 0xee, 0xe6, 0x2a, 0x66, 0xb7, 0xbd, 0x87, 0x6d, 0xe6, 0x0a, 0x20, 0x17, 0x38, 0x75, + 0x9b, 0x12, 0x79, 0x44, 0x86, 0xdd, 0x53, 0xe8, 0x3e, 0x11, 0x7f, 0x35, 0x50, 0x52, 0x95, 0x50, + 0xa4, 0x7b, 0x70, 0x99, 0xd4, 0xb3, 0x6a, 0x6a, 0x76, 0xaf, 0x43, 0x24, 0x3f, 0xa5, 0x7f, 0x1d, + 0x24, 0x86, 0xce, 0x69, 0xe8, 0xf1, 0x8c, 0xf4, 0xc5, 0x34, 0x14, 0xaa, 0x27, 0x1d, 0xcb, 0x4e, + 0x74, 0x57, 0x67, 0x15, 0xa6, 0x39, 0xf0, 0x1d, 0x71, 0x14, 0xdb, 0x37, 0x03, 0x79, 0xe7, 0xd0, + 0x9c, 0x11, 0x3d, 0xf7, 0x03, 0x53, 0x8f, 0x70, 0x8f, 0x58, 0xf5, 0x33, 0xe3, 0x58, 0xf5, 0x73, + 0x1c, 0x4b, 0xe7, 0x59, 0x1c, 0xe9, 0x06, 0xee, 0x35, 0x1b, 0x32, 0x8f, 0x8d, 0x25, 0x0f, 0x0e, + 0xda, 0x86, 0x7c, 0xfb, 0x58, 0xd3, 0x94, 0x7d, 0xa3, 0xe5, 0x72, 0x5f, 0xb5, 0x78, 0xb7, 0xea, + 0xad, 0xe7, 0x95, 0xca, 0x63, 0x9a, 0x89, 0xf9, 0x78, 0x05, 0xcf, 0x32, 0x10, 0x09, 0xec, 0x37, + 0x5a, 0x05, 0x60, 0x8e, 0x9f, 0xd4, 0x5f, 0x28, 0x7d, 0x81, 0x13, 0x2e, 0xca, 0x46, 0x83, 0x22, + 0x24, 0x28, 0x30, 0xf7, 0xa4, 0xb6, 0xc1, 0xa2, 0x92, 0x0b, 0x6c, 0x1b, 0x8e, 0x12, 0xb7, 0x0c, + 0x1a, 0x91, 0xfc, 0x2d, 0xe0, 0x41, 0x37, 0x8a, 0xe3, 0x85, 0xd0, 0xad, 0x16, 0xce, 0x4e, 0x97, + 0x72, 0x32, 0xa5, 0x36, 0x1a, 0x4d, 0x39, 0xc7, 0x32, 0x34, 0x1c, 0xf7, 0x22, 0x61, 0x16, 0xd3, + 0xe3, 0x87, 0x59, 0xfc, 0x0d, 0x01, 0xae, 0xf0, 0xf6, 0x51, 0xf6, 0xa8, 0x6b, 0xb9, 0xda, 0x32, + 0xdc, 0x9e, 0x72, 0x74, 0x3c, 0x9f, 0xa5, 0xe6, 0xd6, 0x2f, 0xc6, 0xb6, 0x73, 0xa8, 0x7b, 0x95, + 0xbd, 0xd6, 0xee, 0x6d, 0x72, 0xe6, 0x8d, 0xe3, 0xaa, 0xe9, 0xda, 0xbd, 0xd5, 0xab, 0x67, 0xa7, + 0x4b, 0x73, 0x83, 0xa9, 0xcf, 0xe5, 0x39, 0x67, 0x90, 0x05, 0xd5, 0x00, 0xb0, 0xdf, 0xbd, 0xe9, + 0xec, 0x19, 0xbf, 0x6c, 0xc6, 0x8e, 0x03, 0x39, 0xc4, 0x8b, 0xee, 0x82, 0xc8, 0xc3, 0x5a, 0xf6, + 0x8d, 0x16, 0x56, 0x1c, 0xe3, 0x53, 0x4c, 0xe7, 0xd9, 0xb4, 0x5c, 0x64, 0x74, 0x22, 0xa2, 0x61, + 0x7c, 0x8a, 0xd1, 0x03, 0xb8, 0x1c, 0xb4, 0x80, 0xb2, 0x87, 0x5b, 0xd6, 0x2b, 0x96, 0x3d, 0x4f, + 0xb3, 0x23, 0x5f, 0xfb, 0xab, 0x24, 0x89, 0xb0, 0x2c, 0xfc, 0x10, 0xe6, 0x87, 0x55, 0x38, 0x3c, + 0x18, 0x73, 0xec, 0xac, 0xf4, 0xbd, 0xe8, 0x46, 0xc9, 0x18, 0x83, 0x86, 0x6f, 0x96, 0xbc, 0x9f, + 0x7a, 0x4f, 0x90, 0xfe, 0x7e, 0x0a, 0x0a, 0xab, 0xdd, 0xd6, 0xd1, 0xd3, 0x4e, 0x83, 0x5d, 0x09, + 0x80, 0xae, 0x41, 0x4e, 0x57, 0x5d, 0x95, 0x15, 0x52, 0x60, 0xe1, 0x6d, 0x84, 0x40, 0x6b, 0x73, + 0x07, 0x4a, 0x21, 0x3f, 0x14, 0xee, 0x6d, 0x4f, 0xab, 0x1d, 0x90, 0xa9, 0x43, 0xfc, 0x7b, 0x30, + 0x1f, 0xca, 0x48, 0x77, 0x35, 0x14, 0x6c, 0xba, 0xb6, 0x81, 0xd9, 0xce, 0x5c, 0x5a, 0x0e, 0x39, + 0xcb, 0xd4, 0x49, 0x72, 0x95, 0xa5, 0xa2, 0x26, 0xcc, 0x90, 0x8c, 0x3d, 0x85, 0xce, 0xc0, 0xde, + 0xce, 0xe9, 0x83, 0x98, 0x6a, 0x45, 0xca, 0x5d, 0xa6, 0xfa, 0xa9, 0x50, 0x1e, 0xfa, 0x53, 0xce, + 0xe3, 0x80, 0xb2, 0xf0, 0x11, 0x88, 0xfd, 0x19, 0xc2, 0xba, 0xcc, 0x30, 0x5d, 0x5e, 0x0a, 0xeb, + 0x32, 0x1d, 0xd2, 0xd3, 0x7a, 0x26, 0x9b, 0x11, 0x27, 0xa5, 0xdf, 0xcc, 0x40, 0xd1, 0xeb, 0x99, + 0x49, 0x9a, 0xf4, 0xab, 0x30, 0x49, 0xfa, 0x91, 0xe7, 0xda, 0x71, 0x7b, 0xc4, 0x80, 0xe0, 0xfe, + 0xdd, 0xa4, 0x7f, 0x79, 0xe8, 0x93, 0xb2, 0x26, 0x31, 0xa5, 0x2c, 0xfc, 0x7e, 0x0a, 0x32, 0xd4, + 0x8a, 0x7e, 0x00, 0x19, 0x7a, 0x27, 0x80, 0x30, 0xf2, 0x4e, 0x00, 0xcf, 0x35, 0x80, 0x64, 0x45, + 0x5b, 0x64, 0x70, 0xe9, 0xde, 0xbc, 0x9b, 0x1b, 0xe7, 0xfd, 0x22, 0x9f, 0x77, 0xb3, 0x55, 0x7a, + 0x7f, 0x42, 0xb3, 0x41, 0x4d, 0x23, 0x36, 0xe3, 0x7a, 0x6b, 0x64, 0x2a, 0x64, 0x28, 0xae, 0x52, + 0x57, 0x25, 0xcb, 0x76, 0xb1, 0xce, 0x8d, 0xde, 0xeb, 0xe7, 0x75, 0x0b, 0xcf, 0xa8, 0xf7, 0xf8, + 0xd0, 0x1b, 0x90, 0x26, 0x53, 0xe1, 0x34, 0xf3, 0x3a, 0x38, 0x3b, 0x5d, 0x4a, 0x93, 0x49, 0x90, + 0xd0, 0xd0, 0x32, 0xe4, 0xa3, 0x93, 0x13, 0xb1, 0x9b, 0xe8, 0x2c, 0x1e, 0x9a, 0x58, 0xa0, 0xe5, + 0x8f, 0x48, 0x06, 0xf8, 0x58, 0xd7, 0xe0, 0xfe, 0x06, 0xbf, 0x26, 0x70, 0xf7, 0xca, 0x86, 0x46, + 0x96, 0x5f, 0x3b, 0xc9, 0xf5, 0xf1, 0x1e, 0x88, 0xb6, 0x6a, 0xea, 0x56, 0xdb, 0xf8, 0x14, 0xb3, + 0x0d, 0x06, 0x87, 0x9f, 0xbc, 0x94, 0x7c, 0x3a, 0xdd, 0x09, 0x70, 0xa4, 0xff, 0x22, 0x70, 0x57, + 0x4c, 0xbf, 0x18, 0xc9, 0x9e, 0x8f, 0xe7, 0xf9, 0xee, 0xa4, 0xb9, 0x6f, 0x79, 0x9e, 0x24, 0x6f, + 0x0e, 0xf3, 0x9b, 0xaa, 0x9b, 0xfb, 0x96, 0x77, 0xd2, 0x67, 0x7b, 0x04, 0x67, 0xe1, 0x97, 0x60, + 0x92, 0x26, 0xbf, 0x46, 0x57, 0xf3, 0xdd, 0x7f, 0x53, 0x62, 0x5a, 0xfa, 0xd3, 0x14, 0xbc, 0x4d, + 0xab, 0xfa, 0x1c, 0xdb, 0xc6, 0x7e, 0x6f, 0xc7, 0xb6, 0x5c, 0xac, 0xb9, 0x58, 0x0f, 0x36, 0xd8, + 0x12, 0x6c, 0x02, 0x1d, 0x72, 0xfc, 0x68, 0xd2, 0xd0, 0xf9, 0x25, 0x20, 0x4f, 0xbe, 0x1a, 0xf0, + 0xce, 0xb2, 0x23, 0xcd, 0xfa, 0x9a, 0x9c, 0x65, 0x92, 0xeb, 0x3a, 0x5a, 0x81, 0x5c, 0xc7, 0xab, + 0xc6, 0x85, 0xbc, 0x5f, 0x7c, 0x2e, 0xb4, 0x01, 0x25, 0x5e, 0x50, 0xb5, 0x65, 0x1c, 0x63, 0x45, + 0x75, 0x2f, 0x32, 0x23, 0x14, 0x18, 0xef, 0x0a, 0x61, 0x5d, 0x71, 0xa5, 0xbf, 0x95, 0x81, 0x5b, + 0xe7, 0xa8, 0x38, 0xc9, 0xee, 0xb5, 0x00, 0xd9, 0x63, 0xf2, 0x22, 0x83, 0xd7, 0x3e, 0x2b, 0xfb, + 0xcf, 0x68, 0x2f, 0xb2, 0xac, 0xec, 0xab, 0x46, 0x8b, 0x2c, 0x43, 0xcc, 0xdf, 0x70, 0xb8, 0x47, + 0x53, 0xbc, 0xff, 0x5e, 0x68, 0x01, 0x7a, 0x4c, 0x05, 0xd1, 0x6c, 0x0e, 0xfa, 0x4c, 0x80, 0x05, + 0xf6, 0x42, 0xe6, 0xf4, 0xd6, 0xf7, 0x9a, 0x0c, 0x7d, 0xcd, 0x5a, 0xcc, 0x6b, 0xc6, 0xd2, 0x51, + 0x39, 0xf4, 0x2e, 0x5e, 0x90, 0xf9, 0xf0, 0xdb, 0xc2, 0x45, 0x59, 0xf8, 0x75, 0x01, 0xf2, 0x21, + 0x02, 0xba, 0x3d, 0x10, 0x62, 0x94, 0x3f, 0x8b, 0x8b, 0x2b, 0xba, 0x35, 0x10, 0x57, 0xb4, 0x9a, + 0xfd, 0xf2, 0x74, 0x29, 0x23, 0x33, 0xd7, 0x75, 0x2f, 0xc2, 0xe8, 0x46, 0x70, 0xa3, 0x4d, 0xba, + 0x2f, 0x93, 0x77, 0xa5, 0x0d, 0x45, 0xa8, 0xaa, 0x77, 0x22, 0x46, 0x11, 0x2a, 0x79, 0x92, 0x7e, + 0x92, 0x82, 0xd9, 0x15, 0x5d, 0x6f, 0x34, 0x28, 0xaa, 0x49, 0x72, 0x8c, 0x21, 0xc8, 0x10, 0x73, + 0x83, 0x87, 0x43, 0xd1, 0xdf, 0xe8, 0x1d, 0x40, 0xba, 0xe1, 0xb0, 0x9b, 0x21, 0x9c, 0x43, 0x55, + 0xb7, 0x5e, 0x05, 0x07, 0xdf, 0xb3, 0x5e, 0x4a, 0xc3, 0x4b, 0x40, 0x0d, 0xa0, 0xb6, 0xb6, 0xe2, + 0xb8, 0xaa, 0xbf, 0xb1, 0x7f, 0x6b, 0xac, 0x00, 0x1b, 0x66, 0x1c, 0xfb, 0x8f, 0x72, 0x8e, 0xc8, + 0xa1, 0x3f, 0x89, 0xc9, 0x67, 0x90, 0x46, 0x71, 0x15, 0xd5, 0xf1, 0x42, 0x43, 0xd8, 0x9d, 0x14, + 0x45, 0x46, 0x5f, 0x71, 0x58, 0xc4, 0x07, 0xf3, 0x28, 0x0f, 0x54, 0x93, 0xe4, 0xb6, 0xec, 0x6f, + 0x0b, 0x50, 0x94, 0xf1, 0xbe, 0x8d, 0x9d, 0xc3, 0x24, 0x75, 0xfe, 0x18, 0x66, 0x6c, 0x26, 0x55, + 0xd9, 0xb7, 0xad, 0xf6, 0x45, 0xe6, 0x8a, 0x3c, 0x67, 0x7c, 0x6c, 0x5b, 0x6d, 0x3e, 0x25, 0x3f, + 0x87, 0x92, 0x5f, 0xc6, 0x24, 0x2b, 0xff, 0xf7, 0x68, 0xf0, 0x28, 0x13, 0x9c, 0xf4, 0x09, 0x74, + 0xb2, 0x1a, 0xa0, 0x5b, 0xf3, 0xe1, 0x82, 0x26, 0xa9, 0x86, 0xff, 0x2c, 0x40, 0xb1, 0xd1, 0xdd, + 0x63, 0x37, 0x1e, 0x25, 0xa7, 0x81, 0x2a, 0xe4, 0x5a, 0x78, 0xdf, 0x55, 0x5e, 0xcb, 0x17, 0x3a, + 0x4b, 0x58, 0xa9, 0x27, 0xf8, 0x13, 0x00, 0x9b, 0x46, 0x4f, 0x51, 0x39, 0xe9, 0x0b, 0xca, 0xc9, + 0x51, 0x5e, 0x42, 0x26, 0xab, 0x4e, 0xc9, 0xaf, 0x66, 0x92, 0xeb, 0xcb, 0x8b, 0xc8, 0xec, 0x90, + 0xbe, 0xc8, 0xec, 0x30, 0xcb, 0x0d, 0xd6, 0xf8, 0x19, 0xa2, 0x0c, 0x73, 0xd4, 0x2c, 0x53, 0xd4, + 0x4e, 0xa7, 0x65, 0x78, 0xb0, 0x87, 0xce, 0x3f, 0x19, 0x79, 0x96, 0x26, 0xad, 0xb0, 0x14, 0x0a, + 0x78, 0xd0, 0x8f, 0x04, 0x98, 0xd9, 0xb7, 0x31, 0xfe, 0x14, 0x2b, 0x74, 0x4a, 0x1e, 0xcf, 0xab, + 0x60, 0x8d, 0x94, 0xe1, 0x2b, 0x9f, 0x3a, 0xe6, 0xd9, 0x8b, 0x1b, 0xe4, 0xbd, 0x68, 0x1b, 0x44, + 0xad, 0xc5, 0xce, 0x41, 0x7d, 0x0f, 0x87, 0xa9, 0xf1, 0x07, 0x40, 0x89, 0x31, 0x07, 0x4e, 0x0e, + 0xcf, 0xc8, 0x60, 0x52, 0x75, 0x85, 0xdf, 0x32, 0x47, 0x8d, 0xed, 0xa8, 0x87, 0x43, 0x38, 0x8a, + 0x3c, 0x74, 0x39, 0x5d, 0x59, 0xc6, 0xaa, 0xce, 0x2d, 0x77, 0x32, 0xae, 0xfc, 0x07, 0x3e, 0xae, + 0x5e, 0xc0, 0x2c, 0xed, 0x37, 0x49, 0x07, 0x83, 0x4a, 0xff, 0x28, 0x0d, 0x28, 0x2c, 0xf9, 0xeb, + 0xeb, 0x6f, 0xa9, 0xe4, 0xfa, 0xdb, 0x3a, 0x48, 0x21, 0x63, 0xa8, 0xa5, 0x3a, 0xae, 0xc2, 0x5c, + 0xe9, 0x1c, 0xa5, 0x83, 0x6d, 0xc5, 0xc1, 0x9a, 0xc5, 0xef, 0x03, 0x12, 0xe4, 0xc5, 0x20, 0xe7, + 0xa6, 0xea, 0xb8, 0xcf, 0x58, 0xbe, 0x1d, 0x6c, 0x37, 0x68, 0x2e, 0xf4, 0x08, 0xae, 0xb4, 0xd5, + 0x93, 0x38, 0xfe, 0x49, 0xca, 0x3f, 0xd7, 0x56, 0x4f, 0x06, 0x98, 0xde, 0x87, 0x85, 0x78, 0x26, + 0xc5, 0xc1, 0xde, 0x51, 0xdb, 0x95, 0x18, 0xc6, 0x06, 0x76, 0xd1, 0x0a, 0x40, 0x00, 0x22, 0xf8, + 0x1a, 0x3d, 0x0e, 0x86, 0xc8, 0xf9, 0x18, 0x42, 0xfa, 0xb1, 0x00, 0xc5, 0x2d, 0xe3, 0xc0, 0x56, + 0x13, 0xbd, 0x6d, 0x07, 0xbd, 0x1f, 0x3d, 0x9b, 0xcc, 0x3f, 0x5c, 0x88, 0xf3, 0x3d, 0x61, 0x39, + 0xbc, 0xfd, 0x47, 0xce, 0x40, 0x96, 0x3e, 0xbf, 0x44, 0x49, 0xce, 0xf9, 0x1a, 0xbc, 0x45, 0xbd, + 0xf9, 0xb8, 0x2b, 0xcf, 0xd7, 0x82, 0x6e, 0xa4, 0xdf, 0x13, 0x60, 0x71, 0xd8, 0x5b, 0x92, 0x1c, + 0x10, 0x32, 0xbd, 0x77, 0x90, 0xbe, 0x41, 0xf1, 0x47, 0xc4, 0x39, 0x33, 0x0d, 0xe2, 0x23, 0x01, + 0xfc, 0xb2, 0x35, 0xe8, 0x8d, 0x84, 0xec, 0xb7, 0x23, 0xfd, 0xeb, 0x05, 0x98, 0xe1, 0xf5, 0xdb, + 0x35, 0x0d, 0xcb, 0x44, 0x0f, 0x20, 0x7d, 0xc0, 0x8f, 0x31, 0xf2, 0xb1, 0x3b, 0xb5, 0xc1, 0x65, + 0x7f, 0xb5, 0x09, 0x99, 0xe4, 0x25, 0x2c, 0x9d, 0xae, 0x1b, 0x53, 0x9e, 0xc0, 0x5f, 0x3d, 0xcc, + 0xd2, 0xe9, 0xba, 0xa8, 0x01, 0x25, 0x2d, 0xb8, 0x61, 0x4c, 0x21, 0xec, 0xe9, 0xa1, 0xdb, 0x8a, + 0xb1, 0x77, 0xbd, 0xd5, 0x26, 0xe4, 0xa2, 0x16, 0x49, 0x40, 0x95, 0xf0, 0xc5, 0x56, 0x99, 0x01, + 0x67, 0xb8, 0x20, 0x2c, 0x3a, 0x7a, 0xa9, 0x56, 0x6d, 0x22, 0x74, 0xff, 0x15, 0x7a, 0x1f, 0xa6, + 0x74, 0x7a, 0x85, 0x12, 0x5f, 0x55, 0xe2, 0x3a, 0x44, 0xe4, 0xa6, 0xaa, 0xda, 0x84, 0xcc, 0x39, + 0xd0, 0x3a, 0xcc, 0xb0, 0x5f, 0x0c, 0xf3, 0xf0, 0xb5, 0xe0, 0xd6, 0x70, 0x09, 0x21, 0x6b, 0xac, + 0x36, 0x21, 0xe7, 0xf5, 0x80, 0x8a, 0x9e, 0x40, 0x5e, 0x6b, 0x61, 0xd5, 0xe6, 0xa2, 0x6e, 0x0f, + 0x8d, 0xe0, 0x1b, 0xb8, 0x76, 0xa9, 0x36, 0x21, 0x83, 0xe6, 0x13, 0x49, 0xa1, 0x6c, 0x7a, 0xfb, + 0x0e, 0x97, 0xf4, 0xee, 0xd0, 0x42, 0x0d, 0x5e, 0x65, 0x54, 0xa3, 0x56, 0x9a, 0x4f, 0x45, 0xdf, + 0x86, 0x8c, 0xa3, 0xa9, 0x26, 0x5f, 0x98, 0x16, 0x87, 0x5c, 0x8f, 0x12, 0x30, 0xd3, 0xdc, 0xe8, + 0x03, 0x06, 0x97, 0xdc, 0x13, 0x6f, 0xef, 0x38, 0x4e, 0xa7, 0x91, 0x30, 0x7c, 0xa2, 0x53, 0x4c, + 0x09, 0x44, 0x0f, 0x2a, 0xc1, 0x87, 0x0a, 0xdd, 0x9e, 0xa7, 0x9b, 0xc5, 0xf1, 0x7a, 0x18, 0x88, + 0x65, 0xae, 0xd1, 0x58, 0x7f, 0x8f, 0x88, 0xb6, 0xa0, 0xc0, 0x04, 0x75, 0x59, 0x98, 0xed, 0xfc, + 0xf2, 0xd0, 0x13, 0xe9, 0x98, 0x40, 0xdf, 0xda, 0x84, 0x3c, 0xa3, 0x86, 0xc8, 0x41, 0xb9, 0xda, + 0xd8, 0x3e, 0x60, 0xbb, 0xd2, 0x23, 0xca, 0x15, 0x76, 0xf3, 0xf3, 0xcb, 0x45, 0x89, 0xe8, 0x57, + 0xe1, 0x12, 0x13, 0xe4, 0x72, 0xef, 0x25, 0xee, 0x04, 0xf3, 0xd6, 0xd0, 0xd3, 0xe4, 0xa1, 0xa1, + 0xb1, 0xb5, 0x09, 0x19, 0xa9, 0x03, 0x89, 0x48, 0x83, 0xcb, 0xec, 0x0d, 0x3c, 0xb6, 0xd2, 0xe6, + 0xe1, 0x80, 0xf3, 0x37, 0xe9, 0x2b, 0xde, 0x19, 0xf6, 0x8a, 0xd8, 0x90, 0xcf, 0xda, 0x84, 0x3c, + 0xa7, 0x0e, 0xa6, 0x06, 0xd5, 0xb0, 0x79, 0x14, 0x1b, 0xef, 0x6e, 0xef, 0x8c, 0xae, 0x46, 0x5c, + 0xf4, 0x9f, 0x5f, 0x8d, 0x48, 0x22, 0x69, 0x40, 0x3f, 0x86, 0x9f, 0x76, 0xa6, 0x99, 0xa1, 0x0d, + 0x18, 0x13, 0xea, 0x46, 0x1a, 0xf0, 0x30, 0x44, 0x46, 0x65, 0x48, 0x1d, 0x68, 0xf4, 0xec, 0x27, + 0x7e, 0x01, 0xf5, 0xc3, 0xb9, 0x6a, 0x13, 0x72, 0xea, 0x40, 0x43, 0x1f, 0x41, 0x96, 0xc5, 0xe6, + 0x9c, 0x98, 0xf3, 0xc5, 0xa1, 0x93, 0x78, 0x34, 0xc2, 0xa9, 0x36, 0x21, 0xd3, 0x70, 0x20, 0xde, + 0x91, 0x79, 0xdc, 0x05, 0x15, 0x51, 0x1e, 0x11, 0x92, 0xdb, 0x17, 0xfd, 0x42, 0x3a, 0x8c, 0xed, + 0x13, 0xd1, 0x0e, 0x14, 0xf9, 0x04, 0xee, 0xf9, 0x91, 0x8b, 0x43, 0xfd, 0x35, 0xe2, 0x5c, 0xc9, + 0x6b, 0x74, 0xa3, 0x2a, 0x44, 0x27, 0x6d, 0x17, 0x95, 0xc8, 0xdb, 0x6e, 0x76, 0x68, 0xdb, 0x0d, + 0x75, 0x6b, 0x26, 0x6d, 0x67, 0x0f, 0x24, 0xa2, 0xef, 0xc2, 0x24, 0x1b, 0x27, 0x88, 0x8a, 0x8c, + 0x73, 0x41, 0xea, 0x1b, 0x22, 0x2c, 0x3f, 0x99, 0xbd, 0x5c, 0xee, 0x9e, 0xa9, 0xb4, 0xac, 0x83, + 0xf9, 0xb9, 0xa1, 0xb3, 0xd7, 0xa0, 0xa3, 0x29, 0x99, 0xbd, 0xdc, 0x80, 0x4a, 0x3a, 0x90, 0xcd, + 0x52, 0xf8, 0x10, 0xbb, 0x34, 0xb4, 0x03, 0xc5, 0x78, 0x6d, 0xd6, 0x68, 0xe0, 0x4c, 0x40, 0xf6, + 0x27, 0x56, 0x07, 0x2b, 0x74, 0x52, 0xbc, 0x3c, 0x7a, 0x62, 0x8d, 0xdc, 0x59, 0xe5, 0x4f, 0xac, + 0x8c, 0x8a, 0x9e, 0x83, 0xc8, 0x2f, 0x4e, 0x51, 0x3c, 0x2f, 0xa2, 0xf9, 0x2b, 0x54, 0xde, 0xbd, + 0xd8, 0x05, 0x31, 0xce, 0xc1, 0xac, 0x46, 0x10, 0x45, 0x34, 0x05, 0x7d, 0x0c, 0xb3, 0x54, 0x9e, + 0xa2, 0x05, 0x77, 0xdd, 0xcc, 0xcf, 0x0f, 0xdc, 0x9c, 0x32, 0xfc, 0x5a, 0x1c, 0x4f, 0xb2, 0xa8, + 0xf5, 0x25, 0x91, 0xf1, 0x60, 0x98, 0x86, 0x4b, 0xd7, 0xee, 0x85, 0xa1, 0xe3, 0x21, 0x7a, 0xcf, + 0x27, 0x19, 0x0f, 0x06, 0xa3, 0x90, 0x6e, 0xdc, 0x37, 0xe3, 0xbd, 0x39, 0xb4, 0x1b, 0x0f, 0x99, + 0xec, 0x0a, 0x6e, 0x64, 0x9e, 0x5b, 0x03, 0x60, 0x38, 0x92, 0x9a, 0xc6, 0x8b, 0x43, 0x0d, 0x80, + 0x7e, 0xaf, 0x4a, 0x62, 0x00, 0xb4, 0x3c, 0x1a, 0x31, 0x00, 0xd8, 0xa1, 0xc7, 0xfc, 0xf5, 0xe1, + 0x8b, 0x55, 0xf8, 0x94, 0x95, 0x2e, 0x56, 0x94, 0x80, 0x56, 0x20, 0x47, 0x8c, 0xfa, 0x1e, 0x1d, + 0xe1, 0x37, 0x86, 0x62, 0xf8, 0xbe, 0x70, 0xab, 0xda, 0x84, 0x9c, 0x7d, 0xc9, 0x49, 0xa4, 0x57, + 0x31, 0x11, 0x7c, 0x6c, 0xdf, 0x1f, 0xda, 0xab, 0x06, 0xe3, 0x6c, 0x48, 0xaf, 0x7a, 0x19, 0x50, + 0x83, 0x25, 0xcf, 0x61, 0xc7, 0x19, 0xf3, 0x6f, 0x8f, 0x5e, 0xf2, 0xa2, 0x87, 0x2f, 0xfe, 0x92, + 0xc7, 0xc9, 0x6c, 0xc9, 0xd3, 0x15, 0xc7, 0xa1, 0xce, 0x1a, 0xf3, 0xb7, 0x46, 0x2c, 0x79, 0x7d, + 0x1b, 0x9c, 0x6c, 0xc9, 0xd3, 0x1b, 0x8c, 0x93, 0x58, 0x7f, 0xb6, 0x77, 0x51, 0x10, 0x87, 0x77, + 0x77, 0x86, 0x5a, 0x7f, 0xb1, 0x37, 0x19, 0x11, 0xeb, 0xcf, 0x8e, 0x24, 0xa0, 0xef, 0xc1, 0x34, + 0xdf, 0x50, 0x9a, 0xbf, 0x3b, 0xc2, 0xc6, 0x0e, 0xef, 0x01, 0x92, 0xee, 0xc8, 0x79, 0xd8, 0xe4, + 0xc0, 0x36, 0xb2, 0xd8, 0xe4, 0x77, 0x6f, 0xc4, 0xe4, 0x30, 0xb0, 0x97, 0xc6, 0x26, 0x87, 0x80, + 0x4c, 0x4a, 0xe3, 0xb0, 0x4d, 0x98, 0xf9, 0x5f, 0x18, 0x5a, 0x9a, 0xe8, 0x6e, 0x14, 0x29, 0x0d, + 0xe7, 0xa1, 0x8b, 0x05, 0x5d, 0xab, 0x99, 0x76, 0xbe, 0x35, 0x7c, 0xb1, 0xe8, 0x87, 0xf5, 0x35, + 0xef, 0xb8, 0x88, 0x69, 0xe5, 0xaf, 0x09, 0x70, 0x9d, 0xf5, 0x01, 0xba, 0x59, 0xde, 0x53, 0xfc, + 0xb3, 0x8e, 0xd0, 0x9e, 0xc5, 0x03, 0x2a, 0xfe, 0xbb, 0x17, 0xdf, 0x9a, 0xf7, 0xde, 0xf8, 0x96, + 0x3a, 0x2a, 0x1f, 0x51, 0x46, 0x9b, 0xa1, 0xbb, 0xf9, 0x87, 0x43, 0x95, 0x11, 0x45, 0xa4, 0x44, + 0x19, 0x9c, 0x07, 0xb5, 0x60, 0x9e, 0x0d, 0x89, 0x00, 0xfd, 0xf8, 0x45, 0x7f, 0x34, 0xd4, 0xff, + 0x71, 0x24, 0xee, 0xab, 0x4d, 0xc8, 0x57, 0x5e, 0xc6, 0x66, 0x58, 0x9d, 0xe6, 0x67, 0xd9, 0x7e, + 0xcc, 0x6c, 0x49, 0x14, 0xd7, 0x33, 0xd9, 0xab, 0xe2, 0xfc, 0x7a, 0x26, 0xfb, 0x86, 0xb8, 0xb0, + 0x9e, 0xc9, 0x5e, 0x13, 0xdf, 0x5c, 0xcf, 0x64, 0x97, 0xc4, 0xeb, 0xeb, 0x99, 0xac, 0x24, 0xde, + 0x94, 0x7e, 0x7b, 0x01, 0x0a, 0x1e, 0x7a, 0x63, 0x28, 0xea, 0x61, 0x18, 0x45, 0x2d, 0x0e, 0x43, + 0x51, 0x1c, 0xef, 0x71, 0x18, 0xf5, 0x30, 0x0c, 0xa3, 0x16, 0x87, 0xc1, 0xa8, 0x80, 0x87, 0xe0, + 0xa8, 0xe6, 0x30, 0x1c, 0x75, 0x6f, 0x0c, 0x1c, 0xe5, 0x8b, 0xea, 0x07, 0x52, 0x6b, 0x83, 0x40, + 0xea, 0xed, 0xd1, 0x40, 0xca, 0x17, 0x15, 0x42, 0x52, 0x1f, 0xf4, 0x21, 0xa9, 0x1b, 0x23, 0x90, + 0x94, 0xcf, 0xef, 0x41, 0xa9, 0x8d, 0x58, 0x28, 0x75, 0xfb, 0x3c, 0x28, 0xe5, 0xcb, 0x89, 0x60, + 0xa9, 0x5a, 0x1c, 0x96, 0xba, 0x75, 0x0e, 0x96, 0xf2, 0x45, 0x85, 0xc1, 0xd4, 0x46, 0x2c, 0x98, + 0xba, 0x7d, 0x1e, 0x98, 0x0a, 0x8a, 0x15, 0x46, 0x53, 0xdf, 0x89, 0xa0, 0xa9, 0xa5, 0xa1, 0x68, + 0xca, 0xe7, 0x66, 0x70, 0xea, 0xc3, 0x7e, 0x38, 0x75, 0x63, 0x04, 0x9c, 0x0a, 0x14, 0xcb, 0xf1, + 0x54, 0x2d, 0x0e, 0x4f, 0xdd, 0x3a, 0x07, 0x4f, 0x05, 0xba, 0x08, 0x01, 0xaa, 0xed, 0x78, 0x40, + 0x75, 0xe7, 0x5c, 0x40, 0xe5, 0x4b, 0x8b, 0x22, 0xaa, 0x5a, 0x1c, 0xa2, 0xba, 0x75, 0x0e, 0xa2, + 0xea, 0x2b, 0x19, 0x83, 0x54, 0xea, 0x48, 0x48, 0xf5, 0xce, 0x98, 0x90, 0xca, 0x17, 0x1d, 0x87, + 0xa9, 0xf4, 0xd1, 0x98, 0xaa, 0x3c, 0x2e, 0xa6, 0xf2, 0x5f, 0x12, 0x0b, 0xaa, 0xd4, 0x91, 0xa0, + 0xea, 0x9d, 0x31, 0x41, 0x55, 0x5f, 0x45, 0xa2, 0xa8, 0x6a, 0x3b, 0x1e, 0x55, 0xdd, 0x39, 0x17, + 0x55, 0x05, 0xad, 0x18, 0x81, 0x55, 0xcb, 0x21, 0x58, 0xf5, 0xd6, 0x10, 0x58, 0xe5, 0xb3, 0x12, + 0x5c, 0xf5, 0xfd, 0x01, 0x5c, 0x25, 0x8d, 0xc2, 0x55, 0x3e, 0xaf, 0x0f, 0xac, 0x6a, 0x71, 0xc0, + 0xea, 0xd6, 0x39, 0xc0, 0x2a, 0xe8, 0x37, 0x21, 0x64, 0xf5, 0x6c, 0x08, 0xb2, 0xba, 0x7b, 0x3e, + 0xb2, 0xf2, 0xe5, 0xf5, 0x41, 0x2b, 0x75, 0x24, 0xb4, 0x7a, 0x67, 0x4c, 0x68, 0x15, 0xb4, 0x60, + 0x0c, 0xb6, 0x7a, 0x2f, 0x8a, 0xad, 0xae, 0x0f, 0xc7, 0x56, 0xbe, 0x18, 0x0e, 0xae, 0x36, 0x62, + 0xc1, 0xd5, 0xed, 0xf3, 0xc0, 0x55, 0x30, 0x9b, 0x85, 0xd1, 0xd5, 0x76, 0x3c, 0xba, 0xba, 0x73, + 0x2e, 0xba, 0x0a, 0x3a, 0x52, 0x04, 0x5e, 0x6d, 0xc4, 0xc2, 0xab, 0xdb, 0xe7, 0xc1, 0xab, 0xbe, + 0xa9, 0x96, 0xe3, 0xab, 0x17, 0x43, 0xf1, 0xd5, 0xfd, 0x71, 0xf0, 0x95, 0x2f, 0x74, 0x00, 0x60, + 0x7d, 0x32, 0x1c, 0x60, 0xfd, 0xc2, 0x05, 0xee, 0x1d, 0x8d, 0x45, 0x58, 0xdf, 0x1f, 0x40, 0x58, + 0xd2, 0x28, 0x84, 0x15, 0x8c, 0x0c, 0x0f, 0x62, 0x55, 0x63, 0x00, 0xd1, 0xdb, 0xa3, 0x01, 0x51, + 0xb0, 0x90, 0x07, 0x88, 0xe8, 0x83, 0x3e, 0x44, 0x74, 0xe3, 0x5c, 0x37, 0xbb, 0x10, 0x24, 0x5a, + 0x1d, 0x84, 0x44, 0x37, 0x47, 0x42, 0x22, 0x5f, 0x42, 0x80, 0x89, 0x36, 0x62, 0x31, 0xd1, 0xed, + 0xf3, 0x30, 0x51, 0xd0, 0x15, 0xc2, 0xa0, 0x68, 0x3b, 0x1e, 0x14, 0xdd, 0x39, 0x17, 0x14, 0xf5, + 0x2d, 0x5b, 0x1e, 0x2a, 0xaa, 0xc5, 0xa1, 0xa2, 0x5b, 0xe7, 0xa0, 0xa2, 0xf0, 0xb2, 0xe5, 0xc3, + 0xa2, 0xe6, 0x30, 0x58, 0x74, 0x6f, 0x0c, 0x58, 0x14, 0x18, 0x73, 0x7d, 0xb8, 0xe8, 0xa3, 0x7e, + 0x5c, 0x24, 0x8d, 0xc2, 0x45, 0x41, 0x27, 0xf2, 0x80, 0xd1, 0x76, 0x3c, 0x30, 0xba, 0x73, 0x2e, + 0x30, 0x0a, 0x8f, 0xeb, 0x10, 0x32, 0xfa, 0xa8, 0x1f, 0x19, 0x49, 0xa3, 0x90, 0x51, 0x50, 0x1e, + 0x0f, 0x1a, 0xd5, 0xe2, 0xa0, 0xd1, 0xad, 0x73, 0xa0, 0x51, 0x68, 0xba, 0x0f, 0xb0, 0xd1, 0x5f, + 0x1f, 0x1f, 0x1b, 0xbd, 0xf7, 0xba, 0x6e, 0x4b, 0xe7, 0x83, 0xa3, 0x8f, 0xfa, 0xc1, 0x91, 0x34, + 0x0a, 0x1c, 0x05, 0xfa, 0xf0, 0xd0, 0x51, 0xfb, 0x5c, 0x74, 0xf4, 0xe0, 0x02, 0xe8, 0xc8, 0x97, + 0xff, 0x95, 0xe1, 0xd1, 0x9b, 0xe2, 0x5b, 0x11, 0x90, 0xf4, 0x2f, 0xb2, 0x30, 0xc5, 0x3f, 0xb9, + 0x15, 0xb9, 0xa6, 0x4a, 0x78, 0x9d, 0x6b, 0xaa, 0xd0, 0xf7, 0xe0, 0x9a, 0xff, 0x40, 0x3d, 0x4f, + 0x14, 0x1e, 0xbc, 0xa3, 0xb5, 0x2c, 0xed, 0x88, 0xae, 0x3d, 0x59, 0x79, 0xde, 0xcf, 0xf2, 0xd8, + 0xb6, 0xda, 0x2c, 0x88, 0x87, 0x9e, 0xe7, 0xa3, 0x35, 0x32, 0x28, 0xa8, 0x91, 0x75, 0xfe, 0xdd, + 0x88, 0x83, 0xb7, 0xf7, 0x71, 0xd6, 0xd7, 0x08, 0x37, 0x47, 0xdf, 0x81, 0x42, 0xd7, 0xc1, 0xb6, + 0xd2, 0xb1, 0x0d, 0xcb, 0x36, 0x5c, 0x16, 0x38, 0x23, 0xac, 0x8a, 0x5f, 0x9e, 0x2e, 0xcd, 0xec, + 0x3a, 0xd8, 0xde, 0xe1, 0x74, 0x79, 0xa6, 0x1b, 0x7a, 0xf2, 0x3e, 0x52, 0x36, 0x39, 0xfe, 0x47, + 0xca, 0x9e, 0x81, 0x48, 0x7d, 0x09, 0xc2, 0xeb, 0x12, 0xbb, 0x51, 0x2a, 0x7e, 0x09, 0x55, 0xf5, + 0xd0, 0xd2, 0x43, 0x6f, 0x96, 0x2a, 0xd9, 0x51, 0x22, 0x7a, 0x02, 0x45, 0xdb, 0xea, 0xd2, 0x8b, + 0x63, 0x3a, 0x56, 0xcb, 0xd0, 0x7a, 0xd4, 0x60, 0x28, 0xc6, 0x9f, 0x99, 0xb2, 0x8c, 0x3b, 0x34, + 0x9f, 0x5c, 0xb0, 0xc3, 0x8f, 0xa8, 0x01, 0xf4, 0xd2, 0x18, 0x4f, 0x0a, 0x1a, 0xb8, 0xa6, 0x7b, + 0xe4, 0x65, 0xf9, 0x2f, 0x54, 0xc3, 0xe5, 0x72, 0xe1, 0x95, 0xff, 0x1b, 0x3d, 0x86, 0x19, 0x7a, + 0x81, 0x31, 0x69, 0x73, 0xab, 0xeb, 0x72, 0x7b, 0xe1, 0x8d, 0x32, 0xfb, 0x94, 0x5d, 0xd9, 0xfb, + 0x94, 0x5d, 0x79, 0x8d, 0x7f, 0xca, 0x8e, 0x1d, 0xb0, 0x7e, 0xfe, 0x1f, 0x97, 0x04, 0x39, 0xef, + 0xf9, 0x78, 0x58, 0x5d, 0x17, 0x3d, 0x80, 0xcb, 0x6d, 0xf5, 0x84, 0xde, 0x83, 0xac, 0x78, 0x06, + 0x0d, 0xbd, 0xf9, 0x8d, 0x7d, 0x3f, 0x0d, 0xb5, 0xd5, 0x13, 0xfa, 0x5d, 0x37, 0x96, 0x44, 0x3f, + 0xca, 0x72, 0x03, 0x66, 0x78, 0x54, 0x03, 0xfb, 0x66, 0x53, 0x89, 0xe6, 0xe4, 0x1f, 0xf0, 0x60, + 0x9f, 0x6d, 0xba, 0x05, 0x45, 0xdd, 0x70, 0x5c, 0xc3, 0xd4, 0x5c, 0x7e, 0xc5, 0x32, 0xbb, 0xa4, + 0xb8, 0xe0, 0x51, 0xd9, 0x3d, 0xca, 0x4d, 0x98, 0xd5, 0x5a, 0x86, 0x6f, 0x26, 0xb2, 0x85, 0x7b, + 0x76, 0xe8, 0xb4, 0x50, 0xa1, 0x79, 0xfb, 0x8f, 0xfa, 0x4b, 0x5a, 0x94, 0x8c, 0x2a, 0x50, 0x3a, + 0x50, 0x5d, 0xfc, 0x4a, 0xed, 0x29, 0x5e, 0xc0, 0x60, 0x9e, 0x06, 0x49, 0x5f, 0x3b, 0x3b, 0x5d, + 0x2a, 0x3c, 0x61, 0x49, 0x03, 0x71, 0x83, 0x85, 0x83, 0x50, 0x82, 0x8e, 0xee, 0x40, 0x49, 0x75, + 0x7a, 0xa6, 0x46, 0x7b, 0x14, 0x36, 0x9d, 0xae, 0xc3, 0x03, 0x67, 0x8a, 0x94, 0x5c, 0xf1, 0xa8, + 0xe8, 0x03, 0x58, 0xe0, 0x5f, 0x52, 0x78, 0xa5, 0xda, 0xba, 0x42, 0x7b, 0x61, 0x30, 0xdc, 0x45, + 0xca, 0x73, 0x95, 0x7d, 0x39, 0x81, 0x64, 0x20, 0x5d, 0x2f, 0x7c, 0x43, 0x31, 0xbb, 0x81, 0x19, + 0xc4, 0xfc, 0x7a, 0x26, 0x3b, 0x23, 0x16, 0xd6, 0x33, 0xd9, 0xa2, 0x58, 0x92, 0x7e, 0x94, 0x86, + 0x12, 0x99, 0x70, 0x1d, 0xc7, 0xb0, 0xcc, 0x9a, 0xef, 0xf0, 0xea, 0x0f, 0x23, 0x81, 0xc6, 0x6e, + 0xf9, 0xcf, 0x68, 0x89, 0xc6, 0xc8, 0x11, 0xc3, 0xd6, 0xff, 0x7e, 0x4a, 0x5a, 0x06, 0x46, 0xa2, + 0x51, 0x40, 0x2b, 0x30, 0xe5, 0x58, 0x5d, 0x5b, 0xf3, 0x6e, 0xf5, 0xbf, 0x37, 0x64, 0x86, 0x0f, + 0xbd, 0xb0, 0xdc, 0xa0, 0x0c, 0x32, 0x67, 0x44, 0x9f, 0x40, 0x89, 0xfd, 0xa2, 0xb1, 0x39, 0x34, + 0x36, 0x86, 0x05, 0x38, 0x3d, 0x18, 0x5b, 0xd6, 0x26, 0x67, 0x94, 0x8b, 0x4e, 0xe4, 0x19, 0x7d, + 0x04, 0x6f, 0x9a, 0x96, 0xd2, 0xc6, 0x6d, 0x8b, 0xcd, 0xec, 0x64, 0x2c, 0xe8, 0x8a, 0xea, 0x2a, + 0xbc, 0xd0, 0xcc, 0x83, 0x72, 0xde, 0xb4, 0xb6, 0x68, 0x16, 0x99, 0xe7, 0x58, 0x71, 0x99, 0x5c, + 0xa9, 0x0c, 0x53, 0xec, 0x17, 0xca, 0xc1, 0xe4, 0xd3, 0x66, 0xad, 0x2a, 0x8b, 0x13, 0x68, 0x06, + 0xb2, 0x8f, 0xe5, 0xa7, 0x5b, 0x4a, 0xe3, 0xd9, 0xa6, 0x28, 0xa0, 0x3c, 0x4c, 0xcb, 0x4f, 0x9f, + 0x36, 0x95, 0x8d, 0xe7, 0x62, 0x4a, 0xba, 0x03, 0xc5, 0x68, 0x89, 0x10, 0xc0, 0x94, 0x5c, 0xdd, + 0x7a, 0x4a, 0xaf, 0xb2, 0xcf, 0xc1, 0xe4, 0xe6, 0xd3, 0xca, 0xca, 0xa6, 0x28, 0x48, 0x7f, 0x26, + 0xc0, 0xcc, 0x2a, 0xfb, 0x36, 0x03, 0xf3, 0xa0, 0xf8, 0xa0, 0xcf, 0xb5, 0xe1, 0x8d, 0x78, 0xe4, + 0x18, 0xef, 0xd2, 0xb0, 0x02, 0x59, 0x3e, 0xc6, 0xbc, 0x18, 0x8e, 0xa5, 0xe1, 0x78, 0x81, 0x6e, + 0xad, 0x79, 0x8e, 0x73, 0x1e, 0x1b, 0x6a, 0x80, 0xa8, 0x7a, 0xba, 0x55, 0x78, 0x49, 0x86, 0xbb, + 0xcf, 0xf5, 0x35, 0x83, 0x37, 0x62, 0xd4, 0x28, 0xf9, 0xfd, 0xcc, 0xe7, 0x3f, 0x5d, 0x9a, 0x90, + 0xfe, 0x3c, 0x03, 0x85, 0xd5, 0xf0, 0x77, 0x28, 0x50, 0xbd, 0xaf, 0xb2, 0x71, 0x56, 0x50, 0x84, + 0xa3, 0x3c, 0xe2, 0x0b, 0x3f, 0xb9, 0xe0, 0xa3, 0x17, 0xac, 0xee, 0xd7, 0x47, 0x78, 0x85, 0x84, + 0x2b, 0x1f, 0x30, 0x2e, 0xfc, 0xbb, 0xb4, 0xbf, 0xaa, 0x96, 0x61, 0x92, 0xc5, 0x28, 0x0a, 0x03, + 0xd7, 0x27, 0xd0, 0xf9, 0x94, 0x58, 0xe9, 0x24, 0x5d, 0x66, 0xd9, 0xc8, 0x2a, 0xdc, 0x7c, 0xad, + 0xcb, 0x22, 0x03, 0xdb, 0xe5, 0xe2, 0xdf, 0xce, 0xec, 0xb2, 0xcb, 0x42, 0xff, 0x1f, 0x7a, 0xec, + 0x91, 0xf7, 0xa1, 0x5f, 0x81, 0x92, 0x66, 0xb5, 0x5a, 0xcc, 0xb8, 0x63, 0xf3, 0xef, 0xe0, 0xf5, + 0x41, 0xb4, 0x08, 0xfc, 0x73, 0xa9, 0x65, 0xff, 0xb3, 0xa9, 0x65, 0x99, 0x7f, 0x36, 0x35, 0x14, + 0x4e, 0x51, 0xf4, 0x85, 0xb1, 0x69, 0xbb, 0x2f, 0xb2, 0x63, 0xfa, 0x75, 0x22, 0x3b, 0x58, 0x3c, + 0x0c, 0xef, 0x79, 0x7f, 0x2c, 0x70, 0xbf, 0xba, 0x4d, 0xcb, 0x3a, 0xea, 0xfa, 0xde, 0x4a, 0x0b, + 0xe1, 0xab, 0x3f, 0x03, 0xa7, 0x73, 0x1a, 0x83, 0x15, 0xb7, 0xe0, 0xa7, 0xbe, 0xda, 0x82, 0x7f, + 0x03, 0x66, 0x3a, 0x36, 0xde, 0xc7, 0xae, 0x76, 0xa8, 0x98, 0xdd, 0x36, 0x0f, 0x40, 0xcb, 0x7b, + 0xb4, 0xed, 0x6e, 0x1b, 0xdd, 0x03, 0xd1, 0xcf, 0xc2, 0x01, 0xb7, 0x77, 0xef, 0x9c, 0x47, 0xe7, + 0xf0, 0x5c, 0xfa, 0x1f, 0x02, 0xcc, 0x45, 0xea, 0xc4, 0xc7, 0xd4, 0x3a, 0xe4, 0x75, 0xdf, 0xc4, + 0x72, 0xe6, 0x85, 0x0b, 0x86, 0x23, 0x84, 0x99, 0x91, 0x02, 0x57, 0xbc, 0xd7, 0xd2, 0x0f, 0x45, + 0x04, 0x62, 0x53, 0x17, 0x14, 0x7b, 0x39, 0x90, 0xb3, 0x16, 0x7a, 0x81, 0x3f, 0xc8, 0xd2, 0x63, + 0x0d, 0x32, 0xe9, 0x7f, 0x0a, 0x20, 0xd2, 0x17, 0x3c, 0xc6, 0x58, 0x4f, 0x64, 0xca, 0xf4, 0xe2, + 0x7e, 0x52, 0xe3, 0x87, 0x98, 0x45, 0x3e, 0x6e, 0x93, 0xee, 0xfb, 0xb8, 0x4d, 0xdc, 0xfc, 0x99, + 0xf9, 0x8a, 0xf3, 0xa7, 0xf4, 0x53, 0x01, 0x8a, 0x7e, 0xb5, 0xd9, 0x57, 0x2d, 0x47, 0x5c, 0x5b, + 0xfb, 0x7a, 0x5f, 0x6e, 0xf4, 0xae, 0xd7, 0x19, 0xeb, 0x43, 0x9b, 0xe1, 0xeb, 0x75, 0xd8, 0x17, + 0x07, 0xff, 0x8e, 0xd7, 0x1d, 0x49, 0x11, 0x2b, 0xc1, 0xbd, 0x26, 0xaf, 0x11, 0xc2, 0xf7, 0x75, + 0x38, 0xe6, 0x3d, 0x0e, 0x29, 0x90, 0xf6, 0x28, 0xa2, 0xa5, 0xb1, 0xe6, 0x77, 0x4f, 0x4b, 0xac, + 0x03, 0xfe, 0x61, 0xb8, 0x25, 0x58, 0x5c, 0xfc, 0x23, 0x48, 0x1f, 0xab, 0xad, 0x51, 0x9e, 0x88, + 0x91, 0x96, 0x93, 0x49, 0x6e, 0xf4, 0x38, 0x72, 0x1d, 0x4c, 0x6a, 0xf8, 0x66, 0xdc, 0xa0, 0x4a, + 0x23, 0xd7, 0xc6, 0x7c, 0x37, 0x3a, 0x80, 0x46, 0xbe, 0x3e, 0x3c, 0x92, 0xde, 0xcf, 0x7c, 0xf1, + 0xd3, 0x25, 0x41, 0xfa, 0x10, 0x10, 0xb1, 0x75, 0xdc, 0x67, 0x5d, 0xcb, 0x0e, 0xae, 0xd6, 0xe9, + 0x8f, 0xef, 0x99, 0x8c, 0x8f, 0xef, 0x91, 0x2e, 0xc3, 0x5c, 0x84, 0x9b, 0xcd, 0x40, 0xd2, 0x77, + 0xe1, 0x8d, 0x27, 0x96, 0xe3, 0x18, 0x9d, 0x46, 0x77, 0x8f, 0x0d, 0x75, 0xb2, 0x5e, 0xf9, 0x73, + 0x6e, 0xb6, 0x43, 0x77, 0x7a, 0x4c, 0x36, 0x37, 0xe5, 0x64, 0xff, 0x59, 0xfa, 0x7d, 0x01, 0xae, + 0x0e, 0x72, 0x32, 0x2d, 0xc7, 0x45, 0x1c, 0x4f, 0x6b, 0x56, 0x70, 0xf3, 0xe3, 0xf9, 0xbd, 0xd5, + 0xcb, 0x4e, 0x6c, 0x6f, 0xfe, 0x4e, 0xa5, 0xad, 0xd2, 0x39, 0x89, 0x5f, 0x44, 0x50, 0xe4, 0xe4, + 0x2d, 0x46, 0x0d, 0xa6, 0xa7, 0xcc, 0x78, 0xd3, 0xd3, 0xff, 0x12, 0x60, 0xb6, 0x89, 0x4d, 0xd5, + 0x74, 0xc9, 0xbc, 0xdf, 0x6d, 0xb3, 0x28, 0xed, 0x12, 0xa4, 0x6d, 0xa5, 0x4b, 0x8b, 0x2e, 0xc8, + 0x29, 0x7b, 0x17, 0xdd, 0x84, 0x02, 0x5d, 0x5b, 0x42, 0xb6, 0x9a, 0x70, 0x37, 0x23, 0x53, 0x6f, + 0x75, 0xd9, 0x33, 0xc4, 0xde, 0x02, 0xa0, 0x99, 0x18, 0x06, 0x4a, 0xd3, 0x1c, 0x39, 0x42, 0xf1, + 0x11, 0x10, 0x8d, 0xfe, 0x09, 0x84, 0x30, 0x07, 0xff, 0x02, 0xa5, 0xfa, 0x52, 0x96, 0x20, 0xcf, + 0xb2, 0x31, 0x31, 0x93, 0x34, 0x0f, 0x50, 0x12, 0x93, 0xf3, 0x18, 0x2e, 0x39, 0x2f, 0x5b, 0x4a, + 0xc7, 0xd2, 0x1d, 0x45, 0xeb, 0x74, 0xb9, 0xe7, 0x34, 0xfb, 0x50, 0xae, 0xb0, 0x7a, 0xf9, 0xec, + 0x74, 0x69, 0xb6, 0xf1, 0x6c, 0x73, 0xc7, 0xd2, 0x9d, 0xca, 0xce, 0x2e, 0xf3, 0x9b, 0x76, 0xe4, + 0x59, 0xe7, 0x65, 0x8b, 0x92, 0x3a, 0x5d, 0x4e, 0x92, 0x3e, 0x4b, 0x01, 0xa2, 0x37, 0x8d, 0xac, + 0xd2, 0xab, 0x3a, 0xbc, 0xe6, 0xb6, 0x60, 0x51, 0x0b, 0x54, 0xa1, 0x38, 0x86, 0x49, 0xec, 0x7b, + 0xd5, 0x71, 0xbd, 0x72, 0xf3, 0x71, 0x13, 0x7b, 0x41, 0x62, 0xbf, 0x26, 0x79, 0x3b, 0x5e, 0x0b, + 0x49, 0x6c, 0x10, 0x81, 0x9b, 0xaa, 0xe3, 0xbf, 0xf0, 0x1e, 0xe4, 0x5c, 0xca, 0xe7, 0xdd, 0xe3, + 0x92, 0x59, 0x9d, 0x39, 0x3b, 0x5d, 0xca, 0x32, 0x61, 0xf5, 0x35, 0x39, 0xcb, 0x92, 0xeb, 0x3a, + 0x5a, 0x86, 0xbc, 0x61, 0x3a, 0xae, 0x4a, 0x8a, 0xc4, 0xb7, 0x1c, 0x0a, 0x2c, 0xd0, 0xb6, 0xce, + 0xc9, 0xf5, 0x35, 0x19, 0xbc, 0x2c, 0x34, 0x9e, 0xad, 0xe8, 0x33, 0xb0, 0x4d, 0x79, 0x1a, 0x7c, + 0x29, 0x17, 0x3c, 0x2a, 0xbb, 0x69, 0xac, 0x01, 0x73, 0x11, 0x4d, 0xf0, 0x85, 0xf9, 0xc3, 0xe8, + 0x84, 0x13, 0xb6, 0x4e, 0xbd, 0xef, 0xec, 0x97, 0xab, 0xa6, 0x66, 0xe9, 0x7c, 0xa4, 0x46, 0x27, + 0x9e, 0x26, 0x94, 0xd6, 0x2d, 0xc3, 0x24, 0xe8, 0xd1, 0xab, 0xea, 0x0a, 0x14, 0xf7, 0x0c, 0x53, + 0xb5, 0x7b, 0x8a, 0xe7, 0x28, 0x2e, 0x9c, 0xe7, 0x28, 0x2e, 0x17, 0x18, 0x07, 0x7f, 0x94, 0x7e, + 0x26, 0x80, 0x18, 0x88, 0xe5, 0x05, 0xfd, 0x16, 0x80, 0xd6, 0xea, 0x3a, 0x2e, 0xb6, 0xbd, 0x09, + 0x60, 0x86, 0x05, 0xa4, 0x55, 0x18, 0xb5, 0xbe, 0x26, 0xe7, 0x78, 0x86, 0xba, 0x8e, 0x6e, 0x46, + 0xaf, 0xcd, 0x99, 0x5c, 0x85, 0xb3, 0x81, 0xcb, 0x72, 0xc8, 0x8c, 0xe2, 0xb8, 0x96, 0xed, 0xeb, + 0x99, 0xcf, 0x28, 0xde, 0x85, 0x62, 0xf4, 0xe2, 0x0c, 0x4c, 0x63, 0x4e, 0x8b, 0xc4, 0xbc, 0x3d, + 0xc6, 0x7e, 0x95, 0x32, 0xe7, 0x57, 0x89, 0x71, 0x78, 0x55, 0xfa, 0x3d, 0x01, 0x4a, 0x15, 0x36, + 0xd0, 0xfd, 0xc9, 0x63, 0xc4, 0x62, 0xb9, 0x06, 0x59, 0xf7, 0xc4, 0x54, 0xda, 0xd8, 0xff, 0x06, + 0xd6, 0x05, 0x6e, 0xfc, 0x9c, 0x76, 0xd9, 0x23, 0xfd, 0xac, 0x2a, 0xdf, 0x08, 0xe1, 0x33, 0xf1, + 0x58, 0x3b, 0x25, 0x3e, 0x13, 0xb3, 0x53, 0xef, 0x37, 0xc8, 0x84, 0x3a, 0x60, 0x49, 0xa2, 0x22, + 0x40, 0xe8, 0xe3, 0x66, 0xfc, 0x33, 0xf2, 0x2b, 0x6b, 0xca, 0xee, 0x76, 0xe5, 0xe9, 0xd6, 0x56, + 0xbd, 0xd9, 0xac, 0xae, 0x89, 0x02, 0x12, 0x61, 0x26, 0xf2, 0x69, 0xb4, 0x14, 0xfb, 0xb0, 0xfc, + 0xfd, 0x77, 0xa0, 0x10, 0xd9, 0x3e, 0x42, 0x25, 0xc8, 0x6f, 0x56, 0x57, 0x1a, 0xd5, 0xda, 0xd3, + 0xcd, 0x35, 0x0a, 0x64, 0xf3, 0x30, 0xbd, 0x5d, 0x5d, 0x91, 0xab, 0x8d, 0xa6, 0x28, 0xdc, 0xff, + 0x4b, 0x00, 0xc1, 0x47, 0x19, 0xc9, 0xab, 0x37, 0xaa, 0x1f, 0x2b, 0xcf, 0x57, 0x36, 0x77, 0xab, + 0x0d, 0x71, 0x02, 0x21, 0x28, 0xae, 0xae, 0x34, 0x2b, 0x35, 0x45, 0xae, 0x36, 0x76, 0x9e, 0x6e, + 0x37, 0xaa, 0xde, 0xf7, 0xeb, 0xef, 0xaf, 0xc1, 0x4c, 0xf8, 0xda, 0x33, 0x34, 0x07, 0xa5, 0x4a, + 0xad, 0x5a, 0xd9, 0x50, 0x9e, 0xd7, 0x57, 0x94, 0x67, 0xbb, 0xd5, 0x5d, 0x82, 0x7c, 0x49, 0x4d, + 0x28, 0xf1, 0xf1, 0xee, 0x26, 0x01, 0xcd, 0x25, 0xc8, 0xb3, 0x67, 0xfa, 0xd5, 0x35, 0x31, 0x75, + 0x7f, 0x0b, 0xf2, 0xa1, 0xeb, 0xd8, 0xc9, 0xeb, 0x76, 0x76, 0x1b, 0x35, 0xa5, 0x59, 0xdf, 0xaa, + 0x36, 0x9a, 0x2b, 0x5b, 0x3b, 0x4c, 0x06, 0xa5, 0xad, 0xac, 0x3e, 0x95, 0x9b, 0xa2, 0xe0, 0x3f, + 0x37, 0x9f, 0xee, 0x56, 0x6a, 0x5e, 0xad, 0xa5, 0x4c, 0x36, 0x2d, 0xa6, 0xef, 0x9f, 0xc0, 0xd5, + 0x21, 0x37, 0x80, 0x91, 0x4a, 0xef, 0x9a, 0xf4, 0x6a, 0x6a, 0x71, 0x02, 0x15, 0x20, 0x47, 0x7a, + 0x2a, 0x0d, 0xaa, 0x17, 0x05, 0x94, 0x85, 0xcc, 0xa1, 0xeb, 0x76, 0xc4, 0x14, 0x9a, 0x82, 0x94, + 0xf3, 0x48, 0x4c, 0x93, 0xff, 0x07, 0x8e, 0x98, 0x21, 0xd0, 0x5d, 0xfd, 0xb4, 0x6b, 0x63, 0x71, + 0x92, 0xc0, 0xff, 0xae, 0x83, 0xed, 0x7d, 0xa3, 0x85, 0xc5, 0x69, 0xc2, 0x62, 0x76, 0x5b, 0x2d, + 0x31, 0x2b, 0x65, 0xb2, 0x53, 0xe2, 0xd4, 0xfd, 0x1b, 0x10, 0xba, 0x74, 0x85, 0xa0, 0xff, 0x4d, + 0xd5, 0xc5, 0x8e, 0x2b, 0x4e, 0xa0, 0x69, 0x48, 0xaf, 0xb4, 0x5a, 0xa2, 0xf0, 0xf0, 0xf3, 0x49, + 0xc8, 0x7a, 0x1f, 0x15, 0x43, 0x9b, 0x30, 0x49, 0x91, 0x2e, 0x5a, 0x1a, 0x8e, 0x81, 0xe9, 0xb0, + 0x5f, 0xb8, 0x7e, 0x1e, 0x48, 0x96, 0x26, 0xd0, 0x5f, 0x86, 0x7c, 0x08, 0x1b, 0xa0, 0xa1, 0xfb, + 0xf9, 0x11, 0x3c, 0xb4, 0x70, 0xfb, 0xbc, 0x6c, 0xbe, 0xfc, 0x17, 0x90, 0xf3, 0xcd, 0x0a, 0x74, + 0x73, 0x94, 0xd1, 0xe1, 0xc9, 0x1e, 0x6d, 0x99, 0x90, 0x51, 0x2a, 0x4d, 0xbc, 0x2b, 0x20, 0x1b, + 0xd0, 0xa0, 0x05, 0x80, 0xe2, 0x7c, 0x66, 0x87, 0x9a, 0x18, 0x0b, 0xf7, 0xc7, 0xca, 0x1d, 0xbc, + 0x93, 0x28, 0x2b, 0x30, 0x63, 0xe2, 0x95, 0x35, 0x60, 0x24, 0xc5, 0x2b, 0x2b, 0xc6, 0x1a, 0xa2, + 0x8d, 0x11, 0x5a, 0x0f, 0x62, 0xe5, 0x0f, 0xae, 0x9c, 0xb1, 0xf2, 0x63, 0x96, 0x15, 0x69, 0x02, + 0x3d, 0x83, 0x0c, 0x99, 0xc3, 0x51, 0x1c, 0xc0, 0xe8, 0x5b, 0x33, 0x16, 0x6e, 0x8e, 0xcc, 0xe3, + 0x89, 0x5c, 0xbd, 0xf7, 0xc5, 0x7f, 0x5a, 0x9c, 0xf8, 0xe2, 0x6c, 0x51, 0xf8, 0xd9, 0xd9, 0xa2, + 0xf0, 0x27, 0x67, 0x8b, 0xc2, 0x9f, 0x9e, 0x2d, 0x0a, 0x3f, 0xfe, 0xf9, 0xe2, 0xc4, 0xcf, 0x7e, + 0xbe, 0x38, 0xf1, 0x27, 0x3f, 0x5f, 0x9c, 0xf8, 0x64, 0x9a, 0x73, 0xef, 0x4d, 0xd1, 0x09, 0xee, + 0xd1, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff, 0xf6, 0x2d, 0x11, 0x9e, 0x53, 0x86, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -12428,6 +12448,26 @@ func (m *ExportRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.SplitMidKey { + i-- + if m.SplitMidKey { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x68 + } + { + size, err := m.ResumeKeyTS.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x62 if m.ReturnSstBelowSize != 0 { i = encodeVarintApi(dAtA, i, uint64(m.ReturnSstBelowSize)) i-- @@ -12674,6 +12714,16 @@ func (m *ExportResponse_File) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + { + size, err := m.EndKeyTS.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a if len(m.LocalityKV) > 0 { i -= len(m.LocalityKV) copy(dAtA[i:], m.LocalityKV) @@ -15707,12 +15757,12 @@ func (m *Header) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n245, err245 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.LockTimeout, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.LockTimeout):]) - if err245 != nil { - return 0, err245 + n247, err247 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.LockTimeout, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.LockTimeout):]) + if err247 != nil { + return 0, err247 } - i -= n245 - i = encodeVarintApi(dAtA, i, uint64(n245)) + i -= n247 + i = encodeVarintApi(dAtA, i, uint64(n247)) i-- dAtA[i] = 0x1 i-- @@ -16853,12 +16903,12 @@ func (m *ContentionEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n275, err275 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) - if err275 != nil { - return 0, err275 + n277, err277 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) + if err277 != nil { + return 0, err277 } - i -= n275 - i = encodeVarintApi(dAtA, i, uint64(n275)) + i -= n277 + i = encodeVarintApi(dAtA, i, uint64(n277)) i-- dAtA[i] = 0x1a { @@ -18352,6 +18402,11 @@ func (m *ExportRequest) Size() (n int) { if m.ReturnSstBelowSize != 0 { n += 1 + sovApi(uint64(m.ReturnSstBelowSize)) } + l = m.ResumeKeyTS.Size() + n += 1 + l + sovApi(uint64(l)) + if m.SplitMidKey { + n += 2 + } return n } @@ -18422,6 +18477,8 @@ func (m *ExportResponse_File) Size() (n int) { if l > 0 { n += 1 + l + sovApi(uint64(l)) } + l = m.EndKeyTS.Size() + n += 1 + l + sovApi(uint64(l)) return n } @@ -31483,6 +31540,59 @@ func (m *ExportRequest) Unmarshal(dAtA []byte) error { break } } + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResumeKeyTS", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ResumeKeyTS.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SplitMidKey", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.SplitMidKey = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -32053,6 +32163,39 @@ func (m *ExportResponse_File) Unmarshal(dAtA []byte) error { } m.LocalityKV = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EndKeyTS", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.EndKeyTS.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 3708c0d6ee74..27decd240678 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1375,8 +1375,29 @@ message FileEncryptionOptions { message ExportRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; ExternalStorage storage = 2 [(gogoproto.nullable) = false]; - util.hlc.Timestamp start_time = 3 [(gogoproto.nullable) = false]; + + // ResumeKeyTS allows export request to resume at arbitrary key timestamp. This + // value doesn't affect export bounds, but ensures that no keys are skipped or + // duplicated when previous request doesn't complete fully and returned EndKeyTS + // together with resume span. + util.hlc.Timestamp resume_key_ts = 12 [ + (gogoproto.nullable) = false, + (gogoproto.customname) = "ResumeKeyTS" + ]; + + // MVCCFilter determines if request exports all data or latest data as of + // Timestamp specified in request header. If all data is requested StartTime + // is used as a lower bound and header Timestamp as higher bound or exported + // entries. MVCCFilter mvcc_filter = 4 [(gogoproto.customname) = "MVCCFilter"]; + // StartTime is only used whe MVCCFilter is set to All. + util.hlc.Timestamp start_time = 3 [(gogoproto.nullable) = false]; + + // Split large rows in the middle of key sequence. This option will allow + // large history being broken up into target_file_size chunks and prevent + // blowing up on memory usage. This option is only allowed together with + // return_sst since caller should reconstruct full tables. + bool split_mid_key = 13; // Return the exported SST data in the response. bool return_sst = 5 [(gogoproto.customname) = "ReturnSST"]; @@ -1447,6 +1468,10 @@ message ExportResponse { // path. message File { Span span = 1 [(gogoproto.nullable) = false]; + util.hlc.Timestamp end_key_ts = 9 [ + (gogoproto.nullable) = false, + (gogoproto.customname) = "EndKeyTS" + ]; string path = 2; reserved 3; reserved 4; diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 880b6c7c3a53..c2b59b724af3 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "in_mem.go", "intent_interleaving_iter.go", "intent_reader_writer.go", + "min_version.go", "multi_iterator.go", "mvcc.go", "mvcc_incremental_iterator.go", @@ -91,6 +92,7 @@ go_test( "intent_interleaving_iter_test.go", "intent_reader_writer_test.go", "main_test.go", + "min_version_test.go", "multi_iterator_test.go", "mvcc_history_test.go", "mvcc_incremental_iterator_test.go", @@ -141,6 +143,7 @@ go_test( "@com_github_cockroachdb_errors//oserror", "@com_github_cockroachdb_pebble//:pebble", "@com_github_cockroachdb_pebble//vfs", + "@com_github_gogo_protobuf//proto", "@com_github_kr_pretty//:pretty", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index b09b47957b5e..2c0f2e4a1f9d 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -1428,8 +1428,8 @@ func runExportToSst( for i := 0; i < b.N; i++ { startTS := hlc.Timestamp{WallTime: int64(numRevisions / 2)} endTS := hlc.Timestamp{WallTime: int64(numRevisions + 2)} - _, _, err := engine.ExportMVCCToSst(context.Background(), keys.LocalMax, roachpb.KeyMax, startTS, endTS, - exportAllRevisions, 0 /* targetSize */, 0 /* maxSize */, useTBI, noopWriter{}) + _, _, _, err := engine.ExportMVCCToSst(context.Background(), keys.LocalMax, roachpb.KeyMax, startTS, endTS, hlc.Timestamp{}, + exportAllRevisions, 0 /* targetSize */, 0 /* maxSize */, false, useTBI, noopWriter{}) if err != nil { b.Fatal(err) } diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 31f7a768e336..37241f5af0d7 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -386,8 +386,11 @@ type Reader interface { // interval (startTS, endTS]. Passing exportAllRevisions exports // every revision of a key for the interval, otherwise only the latest value // within the interval is exported. Deletions are included if all revisions are - // requested or if the start.Timestamp is non-zero. Returns the bytes of an - // SSTable containing the exported keys, the size of exported data, or an error. + // requested or if the start.Timestamp is non-zero. + // + // firstKeyTS is either empty which represent starting from potential intent + // and continuing to versions or non-empty, which represents starting from + // particular version. firstKeyTS will always be empty when !stopMidKey // // If targetSize is positive, it indicates that the export should produce SSTs // which are roughly target size. Specifically, it will return an SST such that @@ -400,6 +403,11 @@ type Reader interface { // to an SST that exceeds maxSize, an error will be returned. This parameter // exists to prevent creating SSTs which are too large to be used. // + // If stopMidKey is false, once function reaches targetSize it would continue + // adding all versions until it reaches next key or end of range. If true, it + // would stop immediately when targetSize is reached and return a next versions + // timestamp in resumeTs so that subsequent operation can pass it to firstKeyTs. + // // If useTBI is true, the backing MVCCIncrementalIterator will initialize a // time-bound iterator along with its regular iterator. The TBI will be used // as an optimization to skip over swaths of uninteresting keys i.e. keys @@ -407,12 +415,19 @@ type Reader interface { // // This function looks at MVCC versions and intents, and returns an error if an // intent is found. + // + // Data is written to dest as it is collected. If error is returned content of + // dest is undefined. + // + // Returns summary containing number of exported bytes, resumeKey and resumeTS + // that allow resuming export if it was cut short because it reached limits or + // an error if export failed for some reason. ExportMVCCToSst( - ctx context.Context, startKey, endKey roachpb.Key, startTS, endTS hlc.Timestamp, - exportAllRevisions bool, targetSize uint64, maxSize uint64, useTBI bool, + ctx context.Context, startKey, endKey roachpb.Key, startTS, endTS, firstKeyTS hlc.Timestamp, + exportAllRevisions bool, targetSize uint64, maxSize uint64, stopMidKey bool, useTBI bool, dest io.Writer, - ) (_ roachpb.BulkOpSummary, resumeKey roachpb.Key, _ error) - // Get returns the value for the given key, nil otherwise. Semantically, it + ) (_ roachpb.BulkOpSummary, resumeKey roachpb.Key, resumeTS hlc.Timestamp, _ error) + // MVCCGet returns the value for the given key, nil otherwise. Semantically, it // behaves as if an iterator with MVCCKeyAndIntentsIterKind was used. // // Deprecated: use storage.MVCCGet instead. diff --git a/pkg/storage/min_version.go b/pkg/storage/min_version.go new file mode 100644 index 000000000000..5696a1771a5b --- /dev/null +++ b/pkg/storage/min_version.go @@ -0,0 +1,92 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "io/ioutil" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/errors/oserror" + "github.com/cockroachdb/pebble/vfs" +) + +// MinVersionFilename is the name of the file containing a marshaled +// roachpb.Version that can be updated during storage-related migrations +// and checked on startup to determine if we can safely use a +// backwards-incompatible feature. +const MinVersionFilename = "STORAGE_MIN_VERSION" + +// WriteMinVersionFile writes the provided version to disk. The caller must +// guarantee that the version will never be downgraded below the given version. +func WriteMinVersionFile(fs vfs.FS, dir string, version *roachpb.Version) error { + if version == nil { + return errors.New("min version should not be nil") + } + ok, err := MinVersionIsAtLeastTargetVersion(fs, dir, version) + if err != nil { + return err + } + if ok { + return nil + } + b, err := protoutil.Marshal(version) + if err != nil { + return err + } + filename := fs.PathJoin(dir, MinVersionFilename) + if err := SafeWriteToFile(fs, dir, filename, b); err != nil { + return err + } + return nil +} + +// MinVersionIsAtLeastTargetVersion returns whether the min version recorded +// on disk is at least the target version. +func MinVersionIsAtLeastTargetVersion( + fs vfs.FS, dir string, target *roachpb.Version, +) (bool, error) { + if target == nil { + return false, errors.New("target version should not be nil") + } + minVersion, err := GetMinVersion(fs, dir) + if err != nil { + return false, err + } + if minVersion == nil { + return false, nil + } + return !minVersion.Less(*target), nil +} + +// GetMinVersion returns the min version recorded on disk if the min version +// file exists and nil otherwise. +func GetMinVersion(fs vfs.FS, dir string) (*roachpb.Version, error) { + filename := fs.PathJoin(dir, MinVersionFilename) + f, err := fs.Open(filename) + if oserror.IsNotExist(err) { + return nil, nil + } + if err != nil { + return nil, err + } + defer f.Close() + b, err := ioutil.ReadAll(f) + if err != nil { + return nil, err + } + version := &roachpb.Version{} + if err := protoutil.Unmarshal(b, version); err != nil { + return nil, err + } + return version, nil +} diff --git a/pkg/storage/min_version_test.go b/pkg/storage/min_version_test.go new file mode 100644 index 000000000000..cd5e32a72ac5 --- /dev/null +++ b/pkg/storage/min_version_test.go @@ -0,0 +1,92 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/pebble/vfs" + "github.com/gogo/protobuf/proto" + "github.com/stretchr/testify/require" +) + +func TestMinVersion(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + version1 := &roachpb.Version{Major: 21, Minor: 1, Patch: 0, Internal: 122} + version2 := &roachpb.Version{Major: 21, Minor: 1, Patch: 0, Internal: 126} + + mem := vfs.NewMem() + dir := "/foo" + require.NoError(t, mem.MkdirAll(dir, os.ModeDir)) + + // Expect nil version when min version file doesn't exist. + v, err := GetMinVersion(mem, dir) + require.NoError(t, err) + require.Nil(t, v) + + // Expect min version to not be at least any target version. + ok, err := MinVersionIsAtLeastTargetVersion(mem, dir, version1) + require.NoError(t, err) + require.False(t, ok) + ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version2) + require.NoError(t, err) + require.False(t, ok) + + // Expect no error when updating min version if no file currently exists. + v = &roachpb.Version{} + proto.Merge(v, version1) + require.NoError(t, WriteMinVersionFile(mem, dir, v)) + + // Expect min version to be version1. + v, err = GetMinVersion(mem, dir) + require.NoError(t, err) + require.True(t, version1.Equal(v)) + + // Expect min version to be at least version1 but not version2. + ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version1) + require.NoError(t, err) + require.True(t, ok) + ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version2) + require.NoError(t, err) + require.False(t, ok) + + // Expect no error when updating min version to a higher version. + v = &roachpb.Version{} + proto.Merge(v, version2) + require.NoError(t, WriteMinVersionFile(mem, dir, v)) + + // Expect min version to be at least version1 and version2. + ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version1) + require.NoError(t, err) + require.True(t, ok) + ok, err = MinVersionIsAtLeastTargetVersion(mem, dir, version2) + require.NoError(t, err) + require.True(t, ok) + + // Expect min version to be version2. + v, err = GetMinVersion(mem, dir) + require.NoError(t, err) + require.True(t, version2.Equal(v)) + + // Expect no-op when trying to update min version to a lower version. + v = &roachpb.Version{} + proto.Merge(v, version1) + require.NoError(t, WriteMinVersionFile(mem, dir, v)) + v, err = GetMinVersion(mem, dir) + require.NoError(t, err) + require.True(t, version2.Equal(v)) +} diff --git a/pkg/storage/mvcc_incremental_iterator.go b/pkg/storage/mvcc_incremental_iterator.go index e127fb0bbdc4..f7c9e75bafc7 100644 --- a/pkg/storage/mvcc_incremental_iterator.go +++ b/pkg/storage/mvcc_incremental_iterator.go @@ -28,6 +28,7 @@ import ( // encountered: // 1. An inline value (non-user data) // 2. An intent whose timestamp lies within the time bounds +// (if not using enableWriteIntentAggregation) // // Note: The endTime is inclusive to be consistent with the non-incremental // iterator, where reads at a given timestamp return writes at that @@ -156,8 +157,8 @@ func NewMVCCIncrementalIterator( } // SeekGE advances the iterator to the first key in the engine which is >= the -// provided key. startKey should be a metadata key to ensure that the iterator -// has a chance to observe any intents on the key if they are there. +// provided key. startKey is not restricted to metadata key and could point to +// any version within a history as required. func (i *MVCCIncrementalIterator) SeekGE(startKey MVCCKey) { if i.timeBoundIter != nil { // Check which is the first key seen by the TBI. diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index ceaa86c626d0..ab85d1aaa6ca 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -153,8 +153,8 @@ func assertExportedErrs( ) { const big = 1 << 30 sstFile := &MemFile{} - _, _, err := e.ExportMVCCToSst(context.Background(), startKey, endKey, startTime, endTime, - revisions, big, big, useTBI, sstFile) + _, _, _, err := e.ExportMVCCToSst(context.Background(), startKey, endKey, startTime, endTime, hlc.Timestamp{}, + revisions, big, big, false, useTBI, sstFile) require.Error(t, err) if intentErr := (*roachpb.WriteIntentError)(nil); errors.As(err, &intentErr) { @@ -182,8 +182,8 @@ func assertExportedKVs( ) { const big = 1 << 30 sstFile := &MemFile{} - _, _, err := e.ExportMVCCToSst(context.Background(), startKey, endKey, startTime, endTime, - revisions, big, big, useTBI, sstFile) + _, _, _, err := e.ExportMVCCToSst(context.Background(), startKey, endKey, startTime, endTime, hlc.Timestamp{}, + revisions, big, big, false, useTBI, sstFile) require.NoError(t, err) data := sstFile.Data() if data == nil { diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 7dcd8ae2c8f2..ea6cd5fef6a5 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -668,18 +668,20 @@ func (p *Pebble) ExportMVCCToSst( ctx context.Context, startKey, endKey roachpb.Key, startTS, endTS hlc.Timestamp, + firstKeyTS hlc.Timestamp, exportAllRevisions bool, targetSize, maxSize uint64, + stopMidKey bool, useTBI bool, dest io.Writer, -) (roachpb.BulkOpSummary, roachpb.Key, error) { +) (roachpb.BulkOpSummary, roachpb.Key, hlc.Timestamp, error) { r := wrapReader(p) // Doing defer r.Free() does not inline. maxIntentCount := MaxIntentsPerWriteIntentError.Get(&p.settings.SV) - summary, k, err := pebbleExportToSst(ctx, r, startKey, endKey, startTS, endTS, exportAllRevisions, - targetSize, maxSize, useTBI, dest, maxIntentCount) + summary, k, err := pebbleExportToSst(ctx, r, MVCCKey{Key: startKey, Timestamp: firstKeyTS}, endKey, startTS, endTS, + exportAllRevisions, targetSize, maxSize, stopMidKey, useTBI, dest, maxIntentCount) r.Free() - return summary, k, err + return summary, k.Key, k.Timestamp, err } // MVCCGet implements the Engine interface. @@ -1389,19 +1391,20 @@ func (p *pebbleReadOnly) ExportMVCCToSst( ctx context.Context, startKey, endKey roachpb.Key, startTS, endTS hlc.Timestamp, + firstKeyTS hlc.Timestamp, exportAllRevisions bool, targetSize, maxSize uint64, + stopMidKey bool, useTBI bool, dest io.Writer, -) (roachpb.BulkOpSummary, roachpb.Key, error) { +) (roachpb.BulkOpSummary, roachpb.Key, hlc.Timestamp, error) { r := wrapReader(p) // Doing defer r.Free() does not inline. maxIntentCount := MaxIntentsPerWriteIntentError.Get(&p.parent.settings.SV) - summary, k, err := pebbleExportToSst( - ctx, r, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize, maxSize, useTBI, dest, - maxIntentCount) + summary, k, err := pebbleExportToSst(ctx, r, MVCCKey{Key: startKey, Timestamp: firstKeyTS}, endKey, startTS, endTS, + exportAllRevisions, targetSize, maxSize, stopMidKey, useTBI, dest, maxIntentCount) r.Free() - return summary, k, err + return summary, k.Key, k.Timestamp, err } func (p *pebbleReadOnly) MVCCGet(key MVCCKey) ([]byte, error) { @@ -1664,19 +1667,20 @@ func (p *pebbleSnapshot) ExportMVCCToSst( ctx context.Context, startKey, endKey roachpb.Key, startTS, endTS hlc.Timestamp, + firstKeyTS hlc.Timestamp, exportAllRevisions bool, targetSize, maxSize uint64, + stopMidKey bool, useTBI bool, dest io.Writer, -) (roachpb.BulkOpSummary, roachpb.Key, error) { +) (roachpb.BulkOpSummary, roachpb.Key, hlc.Timestamp, error) { r := wrapReader(p) // Doing defer r.Free() does not inline. maxIntentCount := MaxIntentsPerWriteIntentError.Get(&p.settings.SV) - summary, k, err := pebbleExportToSst( - ctx, r, startKey, endKey, startTS, endTS, exportAllRevisions, targetSize, maxSize, useTBI, dest, - maxIntentCount) + summary, k, err := pebbleExportToSst(ctx, r, MVCCKey{Key: startKey, Timestamp: firstKeyTS}, endKey, startTS, endTS, + exportAllRevisions, targetSize, maxSize, stopMidKey, useTBI, dest, maxIntentCount) r.Free() - return summary, k, err + return summary, k.Key, k.Timestamp, err } // Get implements the Reader interface. @@ -1797,14 +1801,16 @@ func (e *ExceedMaxSizeError) Error() string { func pebbleExportToSst( ctx context.Context, reader Reader, - startKey, endKey roachpb.Key, + startKey MVCCKey, + endKey roachpb.Key, startTS, endTS hlc.Timestamp, exportAllRevisions bool, targetSize, maxSize uint64, + stopMidKey bool, useTBI bool, dest io.Writer, maxIntentCount int64, -) (roachpb.BulkOpSummary, roachpb.Key, error) { +) (roachpb.BulkOpSummary, MVCCKey, error) { var span *tracing.Span ctx, span = tracing.ChildSpan(ctx, "pebbleExportToSst") _ = ctx // ctx is currently unused, but this new ctx should be used below in the future. @@ -1825,13 +1831,14 @@ func pebbleExportToSst( defer iter.Close() var curKey roachpb.Key // only used if exportAllRevisions var resumeKey roachpb.Key + var resumeTS hlc.Timestamp paginated := targetSize > 0 - for iter.SeekGE(MakeMVCCMetadataKey(startKey)); ; { + for iter.SeekGE(startKey); ; { ok, err := iter.Valid() if err != nil { // This is an underlying iterator error, return it to the caller to deal // with. - return roachpb.BulkOpSummary{}, nil, err + return roachpb.BulkOpSummary{}, MVCCKey{}, err } if !ok { break @@ -1856,29 +1863,32 @@ func pebbleExportToSst( skipTombstones := !exportAllRevisions && startTS.IsEmpty() if len(unsafeValue) > 0 || !skipTombstones { if err := rows.Count(unsafeKey.Key); err != nil { - return roachpb.BulkOpSummary{}, nil, errors.Wrapf(err, "decoding %s", unsafeKey) + return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding %s", unsafeKey) } curSize := rows.BulkOpSummary.DataSize reachedTargetSize := curSize > 0 && uint64(curSize) >= targetSize - if paginated && isNewKey && reachedTargetSize { + if paginated && (isNewKey || stopMidKey) && reachedTargetSize { // Allocate the right size for resumeKey rather than using curKey. resumeKey = append(make(roachpb.Key, 0, len(unsafeKey.Key)), unsafeKey.Key...) + if stopMidKey { + resumeTS = unsafeKey.Timestamp + } break } if unsafeKey.Timestamp.IsEmpty() { // This should never be an intent since the incremental iterator returns // an error when encountering intents. if err := sstWriter.PutUnversioned(unsafeKey.Key, unsafeValue); err != nil { - return roachpb.BulkOpSummary{}, nil, errors.Wrapf(err, "adding key %s", unsafeKey) + return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) } } else { if err := sstWriter.PutMVCC(unsafeKey, unsafeValue); err != nil { - return roachpb.BulkOpSummary{}, nil, errors.Wrapf(err, "adding key %s", unsafeKey) + return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) } } newSize := curSize + int64(len(unsafeKey.Key)+len(unsafeValue)) if maxSize > 0 && newSize > int64(maxSize) { - return roachpb.BulkOpSummary{}, nil, &ExceedMaxSizeError{reached: newSize, maxSize: maxSize} + return roachpb.BulkOpSummary{}, MVCCKey{}, &ExceedMaxSizeError{reached: newSize, maxSize: maxSize} } rows.BulkOpSummary.DataSize = newSize } @@ -1904,18 +1914,18 @@ func pebbleExportToSst( } } err := iter.TryGetIntentError() - return roachpb.BulkOpSummary{}, nil, err + return roachpb.BulkOpSummary{}, MVCCKey{}, err } if rows.BulkOpSummary.DataSize == 0 { // If no records were added to the sstable, skip completing it and return a // nil slice – the export code will discard it anyway (based on 0 DataSize). - return roachpb.BulkOpSummary{}, nil, nil + return roachpb.BulkOpSummary{}, MVCCKey{}, nil } if err := sstWriter.Finish(); err != nil { - return roachpb.BulkOpSummary{}, nil, err + return roachpb.BulkOpSummary{}, MVCCKey{}, err } - return rows.BulkOpSummary, resumeKey, nil + return rows.BulkOpSummary, MVCCKey{Key: resumeKey, Timestamp: resumeTS}, nil } diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index 26cfd05998de..f37a1ed967fb 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -133,11 +133,13 @@ func (p *pebbleBatch) ExportMVCCToSst( ctx context.Context, startKey, endKey roachpb.Key, startTS, endTS hlc.Timestamp, + firstKeyTS hlc.Timestamp, exportAllRevisions bool, targetSize, maxSize uint64, + stopMidKey bool, useTBI bool, dest io.Writer, -) (roachpb.BulkOpSummary, roachpb.Key, error) { +) (roachpb.BulkOpSummary, roachpb.Key, hlc.Timestamp, error) { panic("unimplemented") } diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index ab23ba6e708a..1f637b434631 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -633,8 +633,8 @@ func TestSstExportFailureIntentBatching(t *testing.T) { require.NoError(t, fillInData(ctx, engine, data)) destination := &MemFile{} - _, _, err := engine.ExportMVCCToSst(ctx, key(10), key(20000), ts(999), ts(2000), - true, 0, 0, true, destination) + _, _, _, err := engine.ExportMVCCToSst(ctx, key(10), key(20000), ts(999), ts(2000), hlc.Timestamp{}, + true, 0, 0, false, true, destination) if len(expectedIntentIndices) == 0 { require.NoError(t, err) } else { diff --git a/pkg/ui/cluster-ui/src/breadcrumbs/breadcrumbs.module.scss b/pkg/ui/cluster-ui/src/breadcrumbs/breadcrumbs.module.scss new file mode 100644 index 000000000000..0a1d693cbf76 --- /dev/null +++ b/pkg/ui/cluster-ui/src/breadcrumbs/breadcrumbs.module.scss @@ -0,0 +1,28 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +@import "src/core/index.module"; + +.breadcrumbs { + align-items: center; + display: flex; + margin-bottom: 22px; + + &__item { + &--link { + color: $colors--link; + } + + &--link, &--divider { + margin-right: 10px; + } + } +} + diff --git a/pkg/ui/cluster-ui/src/breadcrumbs/breadcrumbs.tsx b/pkg/ui/cluster-ui/src/breadcrumbs/breadcrumbs.tsx new file mode 100644 index 000000000000..1795667210ba --- /dev/null +++ b/pkg/ui/cluster-ui/src/breadcrumbs/breadcrumbs.tsx @@ -0,0 +1,56 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React, { FunctionComponent, ReactElement } from "react"; +import { Link } from "react-router-dom"; +import classnames from "classnames/bind"; +import styles from "./breadcrumbs.module.scss"; + +export interface BreadcrumbItem { + name: string; + link: string; + onClick?: () => void; +} + +interface BreadcrumbsProps { + items: BreadcrumbItem[]; + divider?: ReactElement; +} + +const cx = classnames.bind(styles); + +export const Breadcrumbs: FunctionComponent = ({ + items, + divider = "/", +}) => { + if (items.length === 0) { + return null; + } + const lastItem = items.pop(); + return ( +
+ {items.map(({ link, name, onClick = () => {} }) => ( +
+ + {name} + + {divider} +
+ ))} +
+ {lastItem?.name} +
+
+ ); +}; diff --git a/pkg/ui/src/views/databases/components/titleWithIcon/titleWithIcon.styl b/pkg/ui/cluster-ui/src/breadcrumbs/index.ts similarity index 73% rename from pkg/ui/src/views/databases/components/titleWithIcon/titleWithIcon.styl rename to pkg/ui/cluster-ui/src/breadcrumbs/index.ts index 08cb2713eb27..ee08e1449f73 100644 --- a/pkg/ui/src/views/databases/components/titleWithIcon/titleWithIcon.styl +++ b/pkg/ui/cluster-ui/src/breadcrumbs/index.ts @@ -1,4 +1,4 @@ -// Copyright 2020 The Cockroach Authors. +// Copyright 2021 The Cockroach Authors. // // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. @@ -8,8 +8,4 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -.title-with-icon - display flex - align-items center - img - margin-right 11px +export * from "./breadcrumbs"; diff --git a/pkg/ui/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.module.scss b/pkg/ui/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.module.scss new file mode 100644 index 000000000000..8d7e5afbe7fd --- /dev/null +++ b/pkg/ui/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.module.scss @@ -0,0 +1,63 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +@import "src/core/index.module"; + +.database-table { + &__col { + &-name { + font-family: $font-family--semi-bold; + } + + &-size { + width: 10em; + } + + &-range-count, &-column-count, &-index-count, &-user-count { + width: 7em; + } + } + + &__no-result { + @include text--body-strong; + } +} + +.icon { + &__container { + display: inline-flex; + align-items: center; + } + + &--md { + height: 24px; + width: 24px; + margin-right: 12px; + } + + &--s { + height: 16px; + width: 16px; + margin-right: 10px; + } + + &--xxs { + height: 8px; + width: 8px; + } + + &--title { + fill: $colors--title; + } + + &--primary { + fill: $colors--primary-text; + } +} diff --git a/pkg/ui/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx b/pkg/ui/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx new file mode 100644 index 000000000000..1885afd38259 --- /dev/null +++ b/pkg/ui/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx @@ -0,0 +1,89 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React from "react"; +import { storiesOf } from "@storybook/react"; +import _ from "lodash"; + +import { withBackground, withRouterProvider } from "src/storybook/decorators"; +import { + randomName, + randomRole, + randomTablePrivilege, +} from "src/storybook/fixtures"; +import { + DatabaseDetailsPage, + DatabaseDetailsPageProps, +} from "./databaseDetailsPage"; + +const withLoadingIndicator: DatabaseDetailsPageProps = { + loading: true, + loaded: false, + name: randomName(), + tables: [], + refreshDatabaseDetails: () => {}, + refreshTableDetails: () => {}, + refreshTableStats: () => {}, +}; + +const withoutData: DatabaseDetailsPageProps = { + loading: false, + loaded: true, + name: randomName(), + tables: [], + refreshDatabaseDetails: () => {}, + refreshTableDetails: () => {}, + refreshTableStats: () => {}, +}; + +const withData: DatabaseDetailsPageProps = { + loading: false, + loaded: true, + name: randomName(), + tables: _.map(Array(42), _item => { + const roles = _.uniq( + _.map(new Array(_.random(1, 3)), _item => randomRole()), + ); + const grants = _.uniq( + _.map(new Array(_.random(1, 5)), _item => randomTablePrivilege()), + ); + + return { + name: randomName(), + details: { + loading: false, + loaded: true, + columnCount: _.random(5, 42), + indexCount: _.random(1, 6), + userCount: roles.length, + roles: roles, + grants: grants, + }, + stats: { + loading: false, + loaded: true, + replicationSizeInBytes: _.random(1000.0) * 1024 ** _.random(1, 2), + rangeCount: _.random(50, 500), + }, + }; + }), + refreshDatabaseDetails: () => {}, + refreshTableDetails: () => {}, + refreshTableStats: () => {}, +}; + +storiesOf("Database Details Page", module) + .addDecorator(withRouterProvider) + .addDecorator(withBackground) + .add("with data", () => ) + .add("without data", () => ) + .add("with loading indicator", () => ( + + )); diff --git a/pkg/ui/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx b/pkg/ui/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx new file mode 100644 index 000000000000..416fed33fcc8 --- /dev/null +++ b/pkg/ui/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx @@ -0,0 +1,410 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React from "react"; +import { Link } from "react-router-dom"; +import { Tooltip } from "antd"; +import classNames from "classnames/bind"; +import _ from "lodash"; + +import { Breadcrumbs } from "src/breadcrumbs"; +import { Dropdown, DropdownOption } from "src/dropdown"; +import { CaretRight } from "src/icon/caretRight"; +import { DatabaseIcon } from "src/icon/databaseIcon"; +import { StackIcon } from "src/icon/stackIcon"; +import { PageConfig, PageConfigItem } from "src/pageConfig"; +import { Pagination, ResultsPerPageLabel } from "src/pagination"; +import { + ColumnDescriptor, + ISortedTablePagination, + SortSetting, + SortedTable, +} from "src/sortedtable"; +import * as format from "src/util/format"; + +import styles from "./databaseDetailsPage.module.scss"; +import sortableTableStyles from "src/sortedtable/sortedtable.module.scss"; +import { + baseHeadingClasses, + statisticsClasses, +} from "src/transactionsPage/transactionsPageClasses"; + +const cx = classNames.bind(styles); +const sortableTableCx = classNames.bind(sortableTableStyles); + +// We break out separate interfaces for some of the nested objects in our data +// both so that they can be available as SortedTable rows and for making +// (typed) test assertions on narrower slices of the data. +// +// The loading and loaded flags help us know when to dispatch the appropriate +// refresh actions. +// +// The overall structure is: +// +// interface DatabaseDetailsPageData { +// loading: boolean; +// loaded: boolean; +// name: string; +// tables: { // DatabaseDetailsPageDataTable[] +// name: string; +// details: { // DatabaseDetailsPageDataTableDetails +// loading: boolean; +// loaded: boolean; +// columnCount: number; +// indexCount: number; +// userCount: number; +// roles: string[]; +// grants: string[]; +// }; +// stats: { // DatabaseDetailsPageDataTableStats +// loading: boolean; +// loaded: boolean; +// replicationSizeInBytes: number; +// rangeCount: number; +// }; +// }[]; +// } +export interface DatabaseDetailsPageData { + loading: boolean; + loaded: boolean; + name: string; + tables: DatabaseDetailsPageDataTable[]; +} + +export interface DatabaseDetailsPageDataTable { + name: string; + details: DatabaseDetailsPageDataTableDetails; + stats: DatabaseDetailsPageDataTableStats; +} + +export interface DatabaseDetailsPageDataTableDetails { + loading: boolean; + loaded: boolean; + columnCount: number; + indexCount: number; + userCount: number; + roles: string[]; + grants: string[]; +} + +export interface DatabaseDetailsPageDataTableStats { + loading: boolean; + loaded: boolean; + replicationSizeInBytes: number; + rangeCount: number; +} + +export interface DatabaseDetailsPageActions { + refreshDatabaseDetails: (database: string) => void; + refreshTableDetails: (database: string, table: string) => void; + refreshTableStats: (database: string, table: string) => void; +} + +export type DatabaseDetailsPageProps = DatabaseDetailsPageData & + DatabaseDetailsPageActions; + +enum ViewMode { + Tables = "Tables", + Grants = "Grants", +} + +interface DatabaseDetailsPageState { + pagination: ISortedTablePagination; + sortSetting: SortSetting; + viewMode: ViewMode; +} + +class DatabaseSortedTable extends SortedTable {} + +export class DatabaseDetailsPage extends React.Component< + DatabaseDetailsPageProps, + DatabaseDetailsPageState +> { + constructor(props: DatabaseDetailsPageProps) { + super(props); + + this.state = { + pagination: { + current: 1, + pageSize: 20, + }, + sortSetting: { + ascending: true, + }, + viewMode: ViewMode.Tables, + }; + } + + componentDidMount() { + this.refresh(); + } + + componentDidUpdate() { + this.refresh(); + } + + private refresh() { + if (!this.props.loaded && !this.props.loading) { + return this.props.refreshDatabaseDetails(this.props.name); + } + + _.forEach(this.props.tables, table => { + if (!table.details.loaded && !table.details.loading) { + return this.props.refreshTableDetails(this.props.name, table.name); + } + + if (!table.stats.loaded && !table.stats.loading) { + return this.props.refreshTableStats(this.props.name, table.name); + } + }); + } + + private changePage(current: number) { + this.setState({ pagination: { ...this.state.pagination, current } }); + } + + private changeSortSetting(sortSetting: SortSetting) { + this.setState({ sortSetting }); + } + + private changeViewMode(viewMode: ViewMode) { + this.setState({ viewMode }); + } + + private columns(): ColumnDescriptor[] { + switch (this.state.viewMode) { + case ViewMode.Tables: + return this.columnsForTablesViewMode(); + case ViewMode.Grants: + return this.columnsForGrantsViewMode(); + default: + throw new Error(`Unknown view mode ${this.state.viewMode}`); + } + } + + private columnsForTablesViewMode(): ColumnDescriptor< + DatabaseDetailsPageDataTable + >[] { + return [ + { + title: ( + + Tables + + ), + cell: table => ( + + + {table.name} + + ), + sort: table => table.name, + className: cx("database-table__col-name"), + name: "name", + }, + { + title: ( + + Replication Size + + ), + cell: table => format.Bytes(table.stats.replicationSizeInBytes), + sort: table => table.stats.replicationSizeInBytes, + className: cx("database-table__col-size"), + name: "replicationSize", + }, + { + title: ( + + Ranges + + ), + cell: table => table.stats.rangeCount, + sort: table => table.stats.rangeCount, + className: cx("database-table__col-range-count"), + name: "rangeCount", + }, + { + title: ( + + Columns + + ), + cell: table => table.details.columnCount, + sort: table => table.details.columnCount, + className: cx("database-table__col-column-count"), + name: "columnCount", + }, + { + title: ( + + Indexes + + ), + cell: table => table.details.indexCount, + sort: table => table.details.indexCount, + className: cx("database-table__col-index-count"), + name: "indexCount", + }, + ]; + } + + private columnsForGrantsViewMode(): ColumnDescriptor< + DatabaseDetailsPageDataTable + >[] { + return [ + { + title: ( + + Tables + + ), + cell: table => ( + + + {table.name} + + ), + sort: table => table.name, + className: cx("database-table__col-name"), + name: "name", + }, + { + title: "Users", + cell: table => table.details.userCount, + sort: table => table.details.userCount, + className: cx("database-table__col-user-count"), + name: "userCount", + }, + { + title: "Roles", + cell: table => _.join(table.details.roles, ", "), + sort: table => _.join(table.details.roles, ", "), + className: cx("database-table__col-roles"), + name: "roles", + }, + { + title: "Grants", + cell: table => _.join(table.details.grants, ", "), + sort: table => _.join(table.details.grants, ", "), + className: cx("database-table__col-grants"), + name: "grants", + }, + ]; + } + + private viewOptions(): DropdownOption[] { + return [ + { + name: "Tables", + value: ViewMode.Tables, + }, + { + name: "Grants", + value: ViewMode.Grants, + }, + ]; + } + + render() { + return ( +
+
+ + } + /> + +

+ + {this.props.name} +

+
+ + + + + View: {this.state.viewMode} + + + + +
+
+

+ +

+
+ + + + This database has no tables. +
+ } + /> + + + + + ); + } +} diff --git a/pkg/ui/cluster-ui/src/databaseDetailsPage/index.tsx b/pkg/ui/cluster-ui/src/databaseDetailsPage/index.tsx new file mode 100644 index 000000000000..01afa0fa3e62 --- /dev/null +++ b/pkg/ui/cluster-ui/src/databaseDetailsPage/index.tsx @@ -0,0 +1,11 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +export * from "./databaseDetailsPage"; diff --git a/pkg/ui/cluster-ui/src/databaseTablePage/databaseTablePage.module.scss b/pkg/ui/cluster-ui/src/databaseTablePage/databaseTablePage.module.scss new file mode 100644 index 000000000000..f11e5503982e --- /dev/null +++ b/pkg/ui/cluster-ui/src/databaseTablePage/databaseTablePage.module.scss @@ -0,0 +1,82 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +@import "src/core/index.module"; + +.database-table-page { + &__indexes { + &--value { + text-align: right; + } + } +} + +.cockroach--tabs { + :global(.ant-tabs-bar) { + border-bottom: 1px solid $grey2; + } + + :global(.ant-tabs-tab) { + @include text--heading-5; + font-weight: $font-weight--medium; + color: $colors--neutral-7; + } + + :global(.ant-tabs-tab-active) { + color: $colors--neutral-8; + } + + :global(.ant-tabs-tab:hover) { + color: $colors--link; + } + + :global(.ant-tabs-ink-bar) { + height: 3px; + border-radius: 40px; + background-color: $colors--link; + } +} + +.summary-card { + h4 { + @include text--body-strong; + color: $colors--neutral-7; + } + + p { + @include text--body; + } +} + +.icon { + &__container { + display: inline-flex; + align-items: center; + } + + &--md { + height: 24px; + width: 24px; + margin-right: 12px; + } + + &--xxs { + height: 8px; + width: 8px; + } + + &--title { + fill: $colors--title; + } + + &--primary { + fill: $colors--primary-text; + } +} \ No newline at end of file diff --git a/pkg/ui/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx b/pkg/ui/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx new file mode 100644 index 000000000000..b2a32174adcf --- /dev/null +++ b/pkg/ui/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx @@ -0,0 +1,90 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React from "react"; +import { storiesOf } from "@storybook/react"; +import _ from "lodash"; + +import { withBackground, withRouterProvider } from "src/storybook/decorators"; +import { + randomName, + randomRole, + randomTablePrivilege, +} from "src/storybook/fixtures"; +import { DatabaseTablePage, DatabaseTablePageProps } from "./databaseTablePage"; + +const withLoadingIndicator: DatabaseTablePageProps = { + databaseName: randomName(), + name: randomName(), + details: { + loading: true, + loaded: false, + createStatement: "", + replicaCount: 0, + indexNames: [], + grants: [], + }, + stats: { + loading: true, + loaded: false, + sizeInBytes: 0, + rangeCount: 0, + }, + refreshTableDetails: () => {}, + refreshTableStats: () => {}, +}; + +const name = randomName(); + +const withData: DatabaseTablePageProps = { + databaseName: randomName(), + name: name, + details: { + loading: false, + loaded: true, + createStatement: ` + CREATE TABLE public.${name} ( + id UUID NOT NULL, + city VARCHAR NOT NULL, + name VARCHAR NULL, + address VARCHAR NULL, + credit_card VARCHAR NULL, + CONSTRAINT "primary" PRIMARY KEY (city ASC, id ASC), + FAMILY "primary" (id, city, name, address, credit_card) + ) + `, + replicaCount: 7, + indexNames: _.map(Array(3), randomName), + grants: _.uniq( + _.map(Array(12), () => { + return { + user: randomRole(), + privilege: randomTablePrivilege(), + }; + }), + ), + }, + stats: { + loading: false, + loaded: true, + sizeInBytes: 44040192, + rangeCount: 4200, + }, + refreshTableDetails: () => {}, + refreshTableStats: () => {}, +}; + +storiesOf("Database Table Page", module) + .addDecorator(withRouterProvider) + .addDecorator(withBackground) + .add("with data", () => ) + .add("with loading indicator", () => ( + + )); diff --git a/pkg/ui/cluster-ui/src/databaseTablePage/databaseTablePage.tsx b/pkg/ui/cluster-ui/src/databaseTablePage/databaseTablePage.tsx new file mode 100644 index 000000000000..1f73b313abc0 --- /dev/null +++ b/pkg/ui/cluster-ui/src/databaseTablePage/databaseTablePage.tsx @@ -0,0 +1,243 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React from "react"; +import { Col, Row, Tabs } from "antd"; +import classNames from "classnames/bind"; +import _ from "lodash"; + +import { Breadcrumbs } from "src/breadcrumbs"; +import { CaretRight } from "src/icon/caretRight"; +import { StackIcon } from "src/icon/stackIcon"; +import { SqlBox } from "src/sql"; +import { ColumnDescriptor, SortSetting, SortedTable } from "src/sortedtable"; +import { SummaryCard, SummaryCardItem } from "src/summaryCard"; +import * as format from "src/util/format"; + +import styles from "./databaseTablePage.module.scss"; +import { baseHeadingClasses } from "src/transactionsPage/transactionsPageClasses"; +const cx = classNames.bind(styles); + +const { TabPane } = Tabs; + +// We break out separate interfaces for some of the nested objects in our data +// so that we can make (typed) test assertions on narrower slices of the data. +// +// The loading and loaded flags help us know when to dispatch the appropriate +// refresh actions. +// +// The overall structure is: +// +// interface DatabaseTablePageData { +// databaseName: string; +// name: string; +// details: { // DatabaseTablePageDataDetails +// loading: boolean; +// loaded: boolean; +// createStatement: string; +// replicaCount: number; +// indexNames: string[]; +// grants: { +// user: string; +// privilege: string; +// }[]; +// }; +// stats: { // DatabaseTablePageDataStats +// loading: boolean; +// loaded: boolean; +// sizeInBytes: number; +// rangeCount: number; +// }; +// } +export interface DatabaseTablePageData { + databaseName: string; + name: string; + details: DatabaseTablePageDataDetails; + stats: DatabaseTablePageDataStats; +} + +export interface DatabaseTablePageDataDetails { + loading: boolean; + loaded: boolean; + createStatement: string; + replicaCount: number; + indexNames: string[]; + grants: Grant[]; +} + +interface Grant { + user: string; + privilege: string; +} + +export interface DatabaseTablePageDataStats { + loading: boolean; + loaded: boolean; + sizeInBytes: number; + rangeCount: number; +} + +export interface DatabaseTablePageActions { + refreshTableDetails: (database: string, table: string) => void; + refreshTableStats: (databse: string, table: string) => void; +} + +export type DatabaseTablePageProps = DatabaseTablePageData & + DatabaseTablePageActions; + +interface DatabaseTablePageState { + sortSetting: SortSetting; +} + +class DatabaseTableGrantsTable extends SortedTable {} + +export class DatabaseTablePage extends React.Component< + DatabaseTablePageProps, + DatabaseTablePageState +> { + constructor(props: DatabaseTablePageProps) { + super(props); + + this.state = { + sortSetting: { + ascending: true, + }, + }; + } + + componentDidMount() { + this.refresh(); + } + + componentDidUpdate() { + this.refresh(); + } + + private refresh() { + if (!this.props.details.loaded && !this.props.details.loading) { + return this.props.refreshTableDetails( + this.props.databaseName, + this.props.name, + ); + } + + if (!this.props.stats.loaded && !this.props.stats.loading) { + return this.props.refreshTableStats( + this.props.databaseName, + this.props.name, + ); + } + } + + private changeSortSetting(sortSetting: SortSetting) { + this.setState({ sortSetting }); + } + + private columns: ColumnDescriptor[] = [ + { + name: "user", + title: "User", + cell: grant => grant.user, + sort: grant => grant.user, + }, + { + name: "privilege", + title: "Grants", + cell: grant => grant.privilege, + sort: grant => grant.privilege, + }, + ]; + + render() { + return ( +
+
+ + } + /> + +

+ + {this.props.name} +

+
+ +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+
+ ); + } +} diff --git a/pkg/ui/cluster-ui/src/databaseTablePage/index.ts b/pkg/ui/cluster-ui/src/databaseTablePage/index.ts new file mode 100644 index 000000000000..8f16c02c3eb4 --- /dev/null +++ b/pkg/ui/cluster-ui/src/databaseTablePage/index.ts @@ -0,0 +1,11 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +export * from "./databaseTablePage"; diff --git a/pkg/ui/cluster-ui/src/databasesPage/databasesPage.module.scss b/pkg/ui/cluster-ui/src/databasesPage/databasesPage.module.scss new file mode 100644 index 000000000000..b7933a5c73ef --- /dev/null +++ b/pkg/ui/cluster-ui/src/databasesPage/databasesPage.module.scss @@ -0,0 +1,63 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +@import "src/core/index.module"; + +.databases-table { + &__col { + &-name { + font-family: $font-family--semi-bold; + } + + &-size { + width: 10em; + } + + &-table-count, &-range-count { + width: 7em; + } + } + + &__no-result { + @include text--body-strong; + } +} + +.icon { + &__container { + display: inline-flex; + align-items: center; + } + + &--md { + height: 24px; + width: 24px; + margin-right: 12px; + } + + &--s { + height: 16px; + width: 16px; + margin-right: 10px; + } + + &--xxs { + height: 8px; + width: 8px; + } + + &--title { + fill: $colors--title; + } + + &--primary { + fill: $colors--primary-text; + } +} \ No newline at end of file diff --git a/pkg/ui/cluster-ui/src/databasesPage/databasesPage.stories.tsx b/pkg/ui/cluster-ui/src/databasesPage/databasesPage.stories.tsx new file mode 100644 index 000000000000..8a981a3859b7 --- /dev/null +++ b/pkg/ui/cluster-ui/src/databasesPage/databasesPage.stories.tsx @@ -0,0 +1,63 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React from "react"; +import { storiesOf } from "@storybook/react"; +import _ from "lodash"; + +import { withBackground, withRouterProvider } from "src/storybook/decorators"; +import { randomName } from "src/storybook/fixtures"; +import { DatabasesPage, DatabasesPageProps } from "./databasesPage"; + +const withLoadingIndicator: DatabasesPageProps = { + loading: true, + loaded: false, + databases: [], + refreshDatabases: () => {}, + refreshDatabaseDetails: () => {}, + refreshTableStats: () => {}, +}; + +const withoutData: DatabasesPageProps = { + loading: false, + loaded: true, + databases: [], + refreshDatabases: () => {}, + refreshDatabaseDetails: () => {}, + refreshTableStats: () => {}, +}; + +const withData: DatabasesPageProps = { + loading: false, + loaded: true, + databases: _.map(Array(42), _item => { + return { + loading: false, + loaded: true, + name: randomName(), + sizeInBytes: _.random(1000.0) * 1024 ** _.random(1, 2), + tableCount: _.random(5, 100), + rangeCount: _.random(50, 500), + missingTables: [], + }; + }), + refreshDatabases: () => {}, + refreshDatabaseDetails: () => {}, + refreshTableStats: () => {}, +}; + +storiesOf("Databases Page", module) + .addDecorator(withRouterProvider) + .addDecorator(withBackground) + .add("with data", () => ) + .add("without data", () => ) + .add("with loading indicator", () => ( + + )); diff --git a/pkg/ui/cluster-ui/src/databasesPage/databasesPage.tsx b/pkg/ui/cluster-ui/src/databasesPage/databasesPage.tsx new file mode 100644 index 000000000000..64c3f4405945 --- /dev/null +++ b/pkg/ui/cluster-ui/src/databasesPage/databasesPage.tsx @@ -0,0 +1,272 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React from "react"; +import { Link } from "react-router-dom"; +import { Tooltip } from "antd"; +import classNames from "classnames/bind"; +import _ from "lodash"; + +import { StackIcon } from "src/icon/stackIcon"; +import { Pagination, ResultsPerPageLabel } from "src/pagination"; +import { + ColumnDescriptor, + ISortedTablePagination, + SortSetting, + SortedTable, +} from "src/sortedtable"; +import * as format from "src/util/format"; + +import styles from "./databasesPage.module.scss"; +import sortableTableStyles from "src/sortedtable/sortedtable.module.scss"; +import { + baseHeadingClasses, + statisticsClasses, +} from "src/transactionsPage/transactionsPageClasses"; + +const cx = classNames.bind(styles); +const sortableTableCx = classNames.bind(sortableTableStyles); + +// We break out separate interfaces for some of the nested objects in our data +// both so that they can be available as SortedTable rows and for making +// (typed) test assertions on narrower slices of the data. +// +// The loading and loaded flags help us know when to dispatch the appropriate +// refresh actions. +// +// The overall structure is: +// +// interface DatabasesPageData { +// loading: boolean; +// loaded: boolean; +// databases: { // DatabasesPageDataDatabase[] +// loading: boolean; +// loaded: boolean; +// name: string; +// sizeInBytes: number; +// tableCount: number; +// rangeCount: number; +// missingTables: { // DatabasesPageDataMissingTable[] +// loading: boolean; +// name: string; +// }[]; +// }[]; +// } +export interface DatabasesPageData { + loading: boolean; + loaded: boolean; + databases: DatabasesPageDataDatabase[]; +} + +export interface DatabasesPageDataDatabase { + loading: boolean; + loaded: boolean; + name: string; + sizeInBytes: number; + tableCount: number; + rangeCount: number; + missingTables: DatabasesPageDataMissingTable[]; +} + +// A "missing" table is one for which we were unable to gather size and range +// count information during the backend call to fetch DatabaseDetails. We +// expose it here so that the component has the opportunity to try to refresh +// those properties for the table directly. +export interface DatabasesPageDataMissingTable { + // Note that we don't need a "loaded" property here because we expect + // the reducers supplying our properties to remove a missing table from + // the list once we've loaded its data. + loading: boolean; + name: string; +} + +export interface DatabasesPageActions { + refreshDatabases: () => void; + refreshDatabaseDetails: (database: string) => void; + refreshTableStats: (database: string, table: string) => void; +} + +export type DatabasesPageProps = DatabasesPageData & DatabasesPageActions; + +interface DatabasesPageState { + pagination: ISortedTablePagination; + sortSetting: SortSetting; +} + +class DatabasesSortedTable extends SortedTable {} + +export class DatabasesPage extends React.Component< + DatabasesPageProps, + DatabasesPageState +> { + constructor(props: DatabasesPageProps) { + super(props); + + this.state = { + pagination: { + current: 1, + pageSize: 20, + }, + sortSetting: { + ascending: true, + columnTitle: null, + }, + }; + } + + componentDidMount() { + this.refresh(); + } + + componentDidUpdate() { + this.refresh(); + } + + private refresh() { + if (!this.props.loaded && !this.props.loading) { + return this.props.refreshDatabases(); + } + + _.forEach(this.props.databases, database => { + if (!database.loaded && !database.loading) { + return this.props.refreshDatabaseDetails(database.name); + } + + _.forEach(database.missingTables, table => { + if (!table.loading) { + return this.props.refreshTableStats(database.name, table.name); + } + }); + }); + } + + private changePage(current: number) { + this.setState({ pagination: { ...this.state.pagination, current } }); + } + + private changeSortSetting(sortSetting: SortSetting) { + this.setState({ sortSetting }); + } + + private columns: ColumnDescriptor[] = [ + { + title: ( + + Databases + + ), + cell: database => ( + + + {database.name} + + ), + sort: database => database.name, + className: cx("databases-table__col-name"), + name: "name", + }, + { + title: ( + + Size + + ), + cell: database => format.Bytes(database.sizeInBytes), + sort: database => database.sizeInBytes, + className: cx("databases-table__col-size"), + name: "size", + }, + { + title: ( + + Tables + + ), + cell: database => database.tableCount, + sort: database => database.tableCount, + className: cx("databases-table__col-table-count"), + name: "tableCount", + }, + { + title: ( + + Range count + + ), + cell: database => database.rangeCount, + sort: database => database.rangeCount, + className: cx("databases-table__col-range-count"), + name: "rangeCount", + }, + ]; + + render() { + return ( +
+
+

Databases

+
+ +
+
+

+ +

+
+ + + + This cluster has no databases. +
+ } + /> + + + + + ); + } +} diff --git a/pkg/ui/cluster-ui/src/databasesPage/index.ts b/pkg/ui/cluster-ui/src/databasesPage/index.ts new file mode 100644 index 000000000000..8e8529654879 --- /dev/null +++ b/pkg/ui/cluster-ui/src/databasesPage/index.ts @@ -0,0 +1,11 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +export * from "./databasesPage"; diff --git a/pkg/ui/cluster-ui/src/icon/caretRight.tsx b/pkg/ui/cluster-ui/src/icon/caretRight.tsx new file mode 100644 index 000000000000..6878378fde0f --- /dev/null +++ b/pkg/ui/cluster-ui/src/icon/caretRight.tsx @@ -0,0 +1,25 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import * as React from "react"; + +interface IconProps { + className?: string; +} + +export const CaretRight = ({ className, ...props }: IconProps) => ( + + + +); diff --git a/pkg/ui/cluster-ui/src/icon/databaseIcon.tsx b/pkg/ui/cluster-ui/src/icon/databaseIcon.tsx index 18020378c275..14edd7c3cc0a 100644 --- a/pkg/ui/cluster-ui/src/icon/databaseIcon.tsx +++ b/pkg/ui/cluster-ui/src/icon/databaseIcon.tsx @@ -11,34 +11,20 @@ import * as React from "react"; interface IconProps { - fill?: string; - size?: number; + className?: string; } -export const DatabaseIcon = ({ fill, size, ...props }: IconProps) => ( - +export const DatabaseIcon = ({ className, ...props }: IconProps) => ( + ); - -DatabaseIcon.defaultProps = { - fill: "#475872", - size: 14, -}; diff --git a/pkg/ui/cluster-ui/src/icon/stackIcon.tsx b/pkg/ui/cluster-ui/src/icon/stackIcon.tsx new file mode 100644 index 000000000000..40dd1d08fbe6 --- /dev/null +++ b/pkg/ui/cluster-ui/src/icon/stackIcon.tsx @@ -0,0 +1,25 @@ +// Copyright 2019 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import * as React from "react"; + +interface IconProps { + className?: string; +} + +export const StackIcon = ({ className, ...props }: IconProps) => ( + + + +); diff --git a/pkg/ui/cluster-ui/src/index.ts b/pkg/ui/cluster-ui/src/index.ts index 356dfb2f70e3..13097a9df947 100644 --- a/pkg/ui/cluster-ui/src/index.ts +++ b/pkg/ui/cluster-ui/src/index.ts @@ -15,6 +15,9 @@ export * from "./anchor"; export * from "./badge"; export * from "./barCharts"; export * from "./button"; +export * from "./databaseDetailsPage"; +export * from "./databaseTablePage"; +export * from "./databasesPage"; export * from "./downloadFile"; export * from "./dropdown"; export * from "./empty"; diff --git a/pkg/ui/cluster-ui/src/sortedtable/sortedtable.tsx b/pkg/ui/cluster-ui/src/sortedtable/sortedtable.tsx index dca777724d34..ea6afc50f22c 100644 --- a/pkg/ui/cluster-ui/src/sortedtable/sortedtable.tsx +++ b/pkg/ui/cluster-ui/src/sortedtable/sortedtable.tsx @@ -379,7 +379,9 @@ export class SortedTable extends React.Component< {loading && } - {count === 0 &&
{renderNoResult}
} + {!loading && count === 0 && ( +
{renderNoResult}
+ )} ); } diff --git a/pkg/ui/cluster-ui/src/storybook/fixtures/index.ts b/pkg/ui/cluster-ui/src/storybook/fixtures/index.ts new file mode 100644 index 000000000000..8462eeda73bf --- /dev/null +++ b/pkg/ui/cluster-ui/src/storybook/fixtures/index.ts @@ -0,0 +1,13 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +export * from "./randomName"; +export * from "./randomPrivilege"; +export * from "./randomRole"; diff --git a/pkg/ui/cluster-ui/src/storybook/fixtures/randomName.ts b/pkg/ui/cluster-ui/src/storybook/fixtures/randomName.ts new file mode 100644 index 000000000000..cd58f427b12a --- /dev/null +++ b/pkg/ui/cluster-ui/src/storybook/fixtures/randomName.ts @@ -0,0 +1,55 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import _ from "lodash"; + +export function randomName(): string { + // Add more! Have fun. + const adjectives = [ + "assured", + "beneficial", + "bulky", + "careless", + "content", + "convincing", + "curious", + "despicable", + "emotional", + "grumpy", + "happy", + "hypnotic", + "joyous", + "kind", + "neglected", + "pathetic", + "personal", + "pompous", + "purple", + "recent", + "ruthless", + "sneezy", + "spacey", + "surly", + ]; + + // Add more! Go for it. + const nouns = [ + "goldfinch", + "hawk", + "hippo", + "moose", + "pteranodon", + "raccoon", + "shark", + "turkey", + ]; + + return _.join([_.sample(adjectives), _.sample(nouns), _.random(1, 42)], "_"); +} diff --git a/pkg/ui/cluster-ui/src/storybook/fixtures/randomPrivilege.ts b/pkg/ui/cluster-ui/src/storybook/fixtures/randomPrivilege.ts new file mode 100644 index 000000000000..a7b531eb441e --- /dev/null +++ b/pkg/ui/cluster-ui/src/storybook/fixtures/randomPrivilege.ts @@ -0,0 +1,24 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import _ from "lodash"; + +export function randomTablePrivilege(): string { + return _.sample([ + "ALL", + "CREATE", + "DROP", + "GRANT", + "SELECT", + "INSERT", + "UPDATE", + "DELETE", + ]); +} diff --git a/pkg/ui/src/views/databases/containers/databases/databases.styl b/pkg/ui/cluster-ui/src/storybook/fixtures/randomRole.ts similarity index 60% rename from pkg/ui/src/views/databases/containers/databases/databases.styl rename to pkg/ui/cluster-ui/src/storybook/fixtures/randomRole.ts index 638057efad9d..67d2e827c48a 100644 --- a/pkg/ui/src/views/databases/containers/databases/databases.styl +++ b/pkg/ui/cluster-ui/src/storybook/fixtures/randomRole.ts @@ -1,4 +1,4 @@ -// Copyright 2018 The Cockroach Authors. +// Copyright 2021 The Cockroach Authors. // // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. @@ -8,14 +8,10 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -.database-summary - margin-bottom 24px +import _ from "lodash"; -hr - border-left none - border-right none - border-bottom none - border-top 1px solid $tooltip-color +import { randomName } from "./randomName"; - margin-top 20px - margin-bottom 20px +export function randomRole(): string { + return _.sample(["root", "admin", "public", randomName()]); +} diff --git a/pkg/ui/src/app.spec.tsx b/pkg/ui/src/app.spec.tsx index c2d6194f21b7..fb72f84bf4c2 100644 --- a/pkg/ui/src/app.spec.tsx +++ b/pkg/ui/src/app.spec.tsx @@ -26,11 +26,9 @@ import { NodeOverview } from "src/views/cluster/containers/nodeOverview"; import { Logs } from "src/views/cluster/containers/nodeLogs"; import { EventPageUnconnected } from "src/views/cluster/containers/events"; import { JobsTable } from "src/views/jobs"; -import { - DatabaseGrantsList, - DatabaseTablesList, -} from "src/views/databases/containers/databases"; -import { TableMain } from "src/views/databases/containers/tableDetails"; +import { DatabasesPage } from "src/views/databases/databasesPage"; +import { DatabaseDetailsPage } from "src/views/databases/databaseDetailsPage"; +import { DatabaseTablePage } from "src/views/databases/databaseTablePage"; import { DataDistributionPage } from "src/views/cluster/containers/dataDistribution"; import { StatementsPage, StatementDetails } from "@cockroachlabs/cluster-ui"; import Debug from "src/views/reports/containers/debug"; @@ -236,29 +234,25 @@ describe("Routing to", () => { /* databases */ } describe("'/databases' path", () => { - it("routes to component", () => { + it("routes to component", () => { navigateToPath("/databases"); - assert.lengthOf(appWrapper.find(DatabaseTablesList), 1); - }); - - it("redirected to '/databases/tables'", () => { - navigateToPath("/databases"); - const location = history.location; - assert.equal(location.pathname, "/databases/tables"); + assert.lengthOf(appWrapper.find(DatabasesPage), 1); }); }); describe("'/databases/tables' path", () => { - it("routes to component", () => { + it("redirected to '/databases'", () => { navigateToPath("/databases/tables"); - assert.lengthOf(appWrapper.find(DatabaseTablesList), 1); + const location = history.location; + assert.equal(location.pathname, "/databases"); }); }); describe("'/databases/grants' path", () => { - it("routes to component", () => { + it("redirected to '/databases'", () => { navigateToPath("/databases/grants"); - assert.lengthOf(appWrapper.find(DatabaseGrantsList), 1); + const location = history.location; + assert.equal(location.pathname, "/databases"); }); }); @@ -275,32 +269,31 @@ describe("Routing to", () => { describe("'/database' path", () => { it("redirected to '/databases'", () => { - navigateToPath("/databases/tables"); + navigateToPath("/database"); const location = history.location; - assert.equal(location.pathname, "/databases/tables"); + assert.equal(location.pathname, "/databases"); }); }); describe("'/database/:${databaseNameAttr}' path", () => { - it("redirected to '/databases'", () => { + it("routes to component", () => { navigateToPath("/database/some-db-name"); - const location = history.location; - assert.equal(location.pathname, "/databases/tables"); + assert.lengthOf(appWrapper.find(DatabaseDetailsPage), 1); }); }); describe("'/database/:${databaseNameAttr}/table' path", () => { - it("redirected to '/databases/tables'", () => { + it("redirected to '/databases/:${databaseNameAttr}'", () => { navigateToPath("/database/some-db-name/table"); const location = history.location; - assert.equal(location.pathname, "/databases/tables"); + assert.equal(location.pathname, "/database/some-db-name"); }); }); describe("'/database/:${databaseNameAttr}/table/:${tableNameAttr}' path", () => { - it("routes to component", () => { + it("routes to component", () => { navigateToPath("/database/some-db-name/table/some-table-name"); - assert.lengthOf(appWrapper.find(TableMain), 1); + assert.lengthOf(appWrapper.find(DatabaseTablePage), 1); }); }); diff --git a/pkg/ui/src/app.tsx b/pkg/ui/src/app.tsx index b7464c9b5120..e774d5d67e52 100644 --- a/pkg/ui/src/app.tsx +++ b/pkg/ui/src/app.tsx @@ -38,11 +38,9 @@ import { EventPage } from "src/views/cluster/containers/events"; import NodeGraphs from "src/views/cluster/containers/nodeGraphs"; import NodeLogs from "src/views/cluster/containers/nodeLogs"; import NodeOverview from "src/views/cluster/containers/nodeOverview"; -import { - DatabaseGrantsList, - DatabaseTablesList, -} from "src/views/databases/containers/databases"; -import TableDetails from "src/views/databases/containers/tableDetails"; +import { DatabasesPage } from "src/views/databases/databasesPage"; +import { DatabaseDetailsPage } from "src/views/databases/databaseDetailsPage"; +import { DatabaseTablePage } from "src/views/databases/databaseTablePage"; import Raft from "src/views/devtools/containers/raft"; import RaftMessages from "src/views/devtools/containers/raftMessages"; import RaftRanges from "src/views/devtools/containers/raftRanges"; @@ -146,35 +144,29 @@ export const App: React.FC = (props: AppProps) => { {/* databases */} - - - + + + - {/* data distribution */} diff --git a/pkg/ui/src/components/tooltip/tooltip.stories.tsx b/pkg/ui/src/components/tooltip/tooltip.stories.tsx index 9628b463f0c7..5e0aef94cd0b 100644 --- a/pkg/ui/src/components/tooltip/tooltip.stories.tsx +++ b/pkg/ui/src/components/tooltip/tooltip.stories.tsx @@ -15,7 +15,6 @@ import * as ClusterTooltips from "src/views/cluster/containers/clusterOverview/t import * as NodeOverviewTooltips from "src/views/cluster/containers/nodeOverview/tooltips"; import * as CapacityArkTooltips from "src/views/clusterviz/components/nodeOrLocality/tooltips"; import * as GraphTooltips from "src/views/cluster/containers/nodeGraphs/dashboards/graphTooltips"; -import * as DataBaseTableTooltips from "src/views/databases/containers/databases/tooltips"; import { ToolTipWrapper } from "src/views/shared/components/toolTip"; import { plainNodeTooltips, @@ -113,6 +112,3 @@ storiesOf("Tooltips/Cluster Overview Page", module) storiesOf("Tooltips/Metrics Page", module) .addDecorator(styledWrapper(wrapperStyle)) .add("Graph tooltips", () => graphTooltipsStack(GraphTooltips)); -storiesOf("Tooltips/Databases Page", module) - .addDecorator(styledWrapper(wrapperStyle)) - .add("Database table tooltips", () => tooltipsStack(DataBaseTableTooltips)); diff --git a/pkg/ui/src/redux/analytics.spec.ts b/pkg/ui/src/redux/analytics.spec.ts index 4b1f23909ae9..0a5627d555b5 100644 --- a/pkg/ui/src/redux/analytics.spec.ts +++ b/pkg/ui/src/redux/analytics.spec.ts @@ -166,12 +166,32 @@ describe("analytics listener", function () { [ testRedaction( - "old database URL", + "old database URL (redirect)", "/databases/database/foobar/table/baz", "/databases/database/[db]/table/[tbl]", ), testRedaction( - "new database URL", + "database URL (redirect)", + "/database/foobar", + "/database/[db]", + ), + testRedaction( + "database tables URL", + "/database/foobar/tables", + "/database/[db]/tables", + ), + testRedaction( + "database grants URL", + "/database/foobar/grants", + "/database/[db]/grants", + ), + testRedaction( + "database table URL (redirect)", + "/database/foobar/table", + "/database/[db]/table", + ), + testRedaction( + "database table URL", "/database/foobar/table/baz", "/database/[db]/table/[tbl]", ), @@ -205,14 +225,25 @@ describe("analytics listener", function () { sync.page(createLocation(input)); assert.isTrue(pageSpy.calledOnce); - assert.deepEqual(pageSpy.args[0][0], { + + const actualArgs = pageSpy.args[0][0]; + const expectedArgs = { userId: clusterID, name: expectedLocation.pathname, properties: { path: expectedLocation.pathname, search: expectedLocation.search, }, - }); + }; + assert.deepEqual( + actualArgs, + expectedArgs, + `Expected:\n${JSON.stringify( + expectedArgs, + null, + 2, + )}\nActual:\n${JSON.stringify(actualArgs, null, 2)}\n`, + ); }); }); }); diff --git a/pkg/ui/src/redux/analytics.ts b/pkg/ui/src/redux/analytics.ts index 939a821a2264..6e2ff77b99ae 100644 --- a/pkg/ui/src/redux/analytics.ts +++ b/pkg/ui/src/redux/analytics.ts @@ -35,14 +35,29 @@ export const defaultRedactions = [ // When viewing a specific database, the database name and table are part of // the URL path. { - match: new RegExp("/databases/database/.*/table/.*"), + match: new RegExp("/databases/database/.+/table/.+"), replace: "/databases/database/[db]/table/[tbl]", }, - // The new URL for a database page. { - match: new RegExp("/database/.*/table/.*"), + match: new RegExp("/database/.+/table/.+"), replace: "/database/[db]/table/[tbl]", }, + { + match: new RegExp("/database/.+/tables"), + replace: "/database/[db]/tables", + }, + { + match: new RegExp("/database/.+/table"), + replace: "/database/[db]/table", + }, + { + match: new RegExp("/database/.+/grants"), + replace: "/database/[db]/grants", + }, + { + match: new RegExp("/database/.+"), + replace: "/database/[db]", + }, // The clusterviz map page, which puts localities in the URL. { match: new RegExp("/overview/map((/.+)+)"), diff --git a/pkg/ui/src/util/api.ts b/pkg/ui/src/util/api.ts index 2e3b3f0c6555..bef30c3cce58 100644 --- a/pkg/ui/src/util/api.ts +++ b/pkg/ui/src/util/api.ts @@ -290,9 +290,11 @@ export function getDatabaseDetails( req: DatabaseDetailsRequestMessage, timeout?: moment.Duration, ): Promise { + const queryString = req.include_stats ? "?include_stats=true" : ""; + return timeoutFetch( serverpb.DatabaseDetailsResponse, - `${API_PREFIX}/databases/${req.database}`, + `${API_PREFIX}/databases/${req.database}${queryString}`, null, timeout, ); diff --git a/pkg/ui/src/util/fakeApi.ts b/pkg/ui/src/util/fakeApi.ts new file mode 100644 index 000000000000..e188b2f7b89b --- /dev/null +++ b/pkg/ui/src/util/fakeApi.ts @@ -0,0 +1,96 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import * as $protobuf from "protobufjs"; + +import { cockroach } from "src/js/protos"; +import { API_PREFIX, toArrayBuffer } from "src/util/api"; +import fetchMock from "src/util/fetch-mock"; + +const { + DatabasesResponse, + DatabaseDetailsResponse, + TableDetailsResponse, + TableStatsResponse, +} = cockroach.server.serverpb; + +// These test-time functions provide typesafe wrappers around fetchMock, +// stubbing HTTP responses from the admin API. +// +// Be sure to call `restore()` after each test that uses `fakeApi`, +// so that your stubbed responses won't leak out to other tests. +// +// Example usage: +// +// describe("The thing I'm testing", function() { +// it("works like this", function() { +// // 1. Set up a fake response from the /databases endpoint. +// fakeApi.stubDatabases({ +// databases: ["one", "two", "three"], +// }); +// +// // 2. Run your code that hits the /databases endpoint. +// // ... +// +// // 3. Assert on its data / behavior. +// assert.deepEqual(myThing.databaseNames(), ["one", "two", "three"]); +// }); +// +// // 4. Tear down any fake responses we set up. +// afterEach(function() { +// fakeApi.restore(); +// }); +// }); + +export function restore() { + fetchMock.restore(); +} + +export function stubDatabases( + response: cockroach.server.serverpb.IDatabasesResponse, +) { + stubGet("/databases", DatabasesResponse.encode(response)); +} + +export function stubDatabaseDetails( + database: string, + response: cockroach.server.serverpb.IDatabaseDetailsResponse, +) { + stubGet( + `/databases/${database}?include_stats=true`, + DatabaseDetailsResponse.encode(response), + ); +} + +export function stubTableDetails( + database: string, + table: string, + response: cockroach.server.serverpb.ITableDetailsResponse, +) { + stubGet( + `/databases/${database}/tables/${table}`, + TableDetailsResponse.encode(response), + ); +} + +export function stubTableStats( + database: string, + table: string, + response: cockroach.server.serverpb.ITableStatsResponse, +) { + stubGet( + `/databases/${database}/tables/${table}/stats`, + TableStatsResponse.encode(response), + ); +} + +function stubGet(path: string, writer: $protobuf.Writer) { + fetchMock.get(`${API_PREFIX}${path}`, toArrayBuffer(writer.finish())); +} diff --git a/pkg/ui/src/views/databases/components/titleWithIcon/titleWithIcon.tsx b/pkg/ui/src/views/databases/components/titleWithIcon/titleWithIcon.tsx deleted file mode 100644 index 51508fd40e16..000000000000 --- a/pkg/ui/src/views/databases/components/titleWithIcon/titleWithIcon.tsx +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import React from "react"; -import "./titleWithIcon.styl"; - -interface TitleWithIconProps { - title: string; - src: string; -} - -const TitleWithIcon: React.FC = ({ title, src }) => ( -

- Stack - {title} -

-); - -export default TitleWithIcon; diff --git a/pkg/ui/src/views/databases/containers/databaseGrants/databaseGrants.fixtures.ts b/pkg/ui/src/views/databases/containers/databaseGrants/databaseGrants.fixtures.ts deleted file mode 100644 index c5638aab9688..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseGrants/databaseGrants.fixtures.ts +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -const mockedActions = { - setSort: () => {}, - refreshDatabaseDetails: () => {}, -}; - -export const loadingProps: any = { - ...mockedActions, - name: "system", - dbResponse: { inFlight: true }, -}; - -export const emptyProps: any = { - ...mockedActions, - name: "system", - dbResponse: { inFlight: false }, -}; - -export const fullfilledProps: any = { - ...mockedActions, - name: "system", - dbResponse: { inFlight: false }, - grants: [ - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["SELECT"] }, - ], -}; diff --git a/pkg/ui/src/views/databases/containers/databaseGrants/databaseGrants.spec.tsx b/pkg/ui/src/views/databases/containers/databaseGrants/databaseGrants.spec.tsx deleted file mode 100644 index 9baffc2b77c2..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseGrants/databaseGrants.spec.tsx +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import React from "react"; -import _ from "lodash"; -import { mount } from "enzyme"; -import { assert } from "chai"; -import classNames from "classnames/bind"; -import { DatabaseSummaryGrants } from "./"; -import { - emptyProps, - loadingProps, - fullfilledProps, -} from "./databaseGrants.fixtures"; - -import styles from "src/views/shared/components/sortabletable/sortabletable.module.styl"; -const cx = classNames.bind(styles); - -describe("", function () { - it("render loading state ", function () { - const wrapper = mount(); - assert.lengthOf(wrapper.find(`.${cx("table__loading")}`), 1); - }); - - it("render fulfilled state ", function () { - const wrapper = mount(); - assert.lengthOf(wrapper.find(`.${cx("sort-table__row--body")}`), 4); - }); - - it("render empty state ", function () { - const wrapper = mount(); - assert.lengthOf(wrapper.find(`.${cx("table__no-results")}`), 1); - }); -}); diff --git a/pkg/ui/src/views/databases/containers/databaseGrants/databaseGrants.stories.tsx b/pkg/ui/src/views/databases/containers/databaseGrants/databaseGrants.stories.tsx deleted file mode 100644 index 1b85d977a8e7..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseGrants/databaseGrants.stories.tsx +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. -import React from "react"; -import { storiesOf } from "@storybook/react"; -import { DatabaseSummaryGrants } from "./"; -import { - emptyProps, - loadingProps, - fullfilledProps, -} from "./databaseGrants.fixtures"; - -storiesOf("DatabaseSummaryGrants", module) - .add("Loading state", () => ) - .add("Empty state", () => ) - .add("Default state", () => ); diff --git a/pkg/ui/src/views/databases/containers/databaseGrants/index.tsx b/pkg/ui/src/views/databases/containers/databaseGrants/index.tsx deleted file mode 100644 index 731ea3768fcb..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseGrants/index.tsx +++ /dev/null @@ -1,129 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import React from "react"; -import { connect } from "react-redux"; -import * as protos from "src/js/protos"; -import { LocalSetting } from "src/redux/localsettings"; -import { AdminUIState } from "src/redux/state"; -import { - databaseDetails, - DatabaseSummaryExplicitData, - grants as selectGrants, - DatabaseSummaryBase, -} from "src/views/databases/containers/databaseSummary"; -import { SortSetting } from "src/views/shared/components/sortabletable"; -import { SortedTable } from "src/views/shared/components/sortedtable"; -import { SummaryCard } from "src/views/shared/components/summaryCard"; -import { DatabaseIcon } from "src/components/icon/databaseIcon"; -import Stack from "assets/stack.svg"; -import { SummaryHeadlineStat } from "src/views/shared/components/summaryBar"; -import TitleWithIcon from "../../components/titleWithIcon/titleWithIcon"; -import { refreshDatabaseDetails } from "src/redux/apiReducers"; -import { privileges } from "src/util/docs"; - -class DatabaseGrantsSortedTable extends SortedTable {} - -const grantsSortSetting = new LocalSetting( - "databases/sort_setting/grants", - (s) => s.localSettings, -); - -// DatabaseSummaryGrants displays a summary section describing the grants -// active on a single database. -export class DatabaseSummaryGrants extends DatabaseSummaryBase { - totalUsers() { - const grants = this.props.grants; - return grants && grants.length; - } - - noDatabaseResults = () => ( - <> -

- - No users have been granted access to this database. -

-

- - Read more about privileges. - -

- - ); - - render() { - const { grants, sortSetting, dbResponse } = this.props; - const dbID = this.props.name; - const loading = dbResponse ? !!dbResponse.inFlight : true; - - return ( -
-
- -
-
-
- this.props.setSort(setting)} - columns={[ - { - title: "User", - cell: (grant) => grant.user, - sort: (grant) => grant.user, - }, - { - title: "Grants", - cell: (grant) => grant.privileges.join(", "), - }, - ]} - loading={loading} - renderNoResult={loading ? undefined : this.noDatabaseResults()} - /> -
-
- - - -
-
-
- ); - } -} - -const mapStateToProps = ( - state: AdminUIState, - ownProps: DatabaseSummaryExplicitData, -) => ({ - // RootState contains declaration for whole state - // tableInfos: tableInfos(state, ownProps.name), - sortSetting: grantsSortSetting.selector(state), - dbResponse: databaseDetails(state)[ownProps.name], - grants: selectGrants(state, ownProps.name), -}); - -const mapDispatchToProps = { - setSort: grantsSortSetting.set, - refreshDatabaseDetails: refreshDatabaseDetails, -}; - -// Connect the DatabaseSummaryGrants class with our redux store. -export default connect( - mapStateToProps, - mapDispatchToProps, -)(DatabaseSummaryGrants as any); diff --git a/pkg/ui/src/views/databases/containers/databaseSummary/index.tsx b/pkg/ui/src/views/databases/containers/databaseSummary/index.tsx deleted file mode 100644 index 4d7b405158d9..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseSummary/index.tsx +++ /dev/null @@ -1,164 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import _ from "lodash"; -import React from "react"; - -import * as protos from "src/js/protos"; - -import { AdminUIState } from "src/redux/state"; -import { - refreshDatabaseDetails, - refreshTableDetails, - refreshTableStats, - generateTableID, - KeyedCachedDataReducerState, -} from "src/redux/apiReducers"; - -import { SortSetting } from "src/views/shared/components/sortabletable"; - -import { TableInfo } from "src/views/databases/data/tableInfo"; - -// DatabaseSummaryImplicitData describes properties which must be explicitly set -// on a DatabaseSummary component. -export interface DatabaseSummaryExplicitData { - name: string; - updateOnLoad?: boolean; -} - -// DatabaseSummaryConnectedData describes properties which are applied to a -// DatabaseSummary component by connecting to a redux store. -interface DatabaseSummaryConnectedData { - sortSetting: SortSetting; - tableInfos: TableInfo[]; - dbResponse: KeyedCachedDataReducerState; - grants: protos.cockroach.server.serverpb.DatabaseDetailsResponse.Grant[]; -} - -// DatabaseSummaryActions describes actions which can be dispatched by a -// DatabaseSummary component. -interface DatabaseSummaryActions { - setSort: (setting: SortSetting) => void; - refreshDatabaseDetails: typeof refreshDatabaseDetails; - refreshTableDetails: typeof refreshTableDetails; - refreshTableStats: typeof refreshTableStats; -} - -export type DatabaseSummaryProps = DatabaseSummaryExplicitData & - DatabaseSummaryConnectedData & - DatabaseSummaryActions; - -interface DatabaseSummaryState { - finishedLoadingTableData: boolean; -} - -// DatabaseSummaryBase implements common lifecycle methods for DatabaseSummary -// components, which differ primarily by their render() method. -// TODO(mrtracy): We need to find a better abstraction for the common -// "refresh-on-mount-or-receiveProps" we have in many of our connected -// components; that would allow us to avoid this inheritance. -export class DatabaseSummaryBase extends React.Component< - DatabaseSummaryProps, - DatabaseSummaryState -> { - // loadTableDetails loads data for each table which have no info in the store. - // TODO(mrtracy): Should this be refreshing data always? Not sure if there - // is a performance concern with invalidation periods. - async loadTableDetails(props = this.props) { - if (props.tableInfos && props.tableInfos.length > 0) { - for (const tblInfo of props.tableInfos) { - // TODO(davidh): this is a stopgap inserted to deal with DBs containing hundreds of tables - await Promise.all([ - _.isUndefined(tblInfo.numColumns) - ? props.refreshTableDetails( - new protos.cockroach.server.serverpb.TableDetailsRequest({ - database: props.name, - table: tblInfo.name, - }), - ) - : null, - _.isUndefined(tblInfo.physicalSize) - ? props.refreshTableStats( - new protos.cockroach.server.serverpb.TableStatsRequest({ - database: props.name, - table: tblInfo.name, - }), - ) - : null, - ]); - } - } - this.setState({ finishedLoadingTableData: true }); - } - - // Refresh when the component is mounted. - async componentDidMount() { - this.props.refreshDatabaseDetails( - new protos.cockroach.server.serverpb.DatabaseDetailsRequest({ - database: this.props.name, - }), - ); - if (this.props.updateOnLoad) { - await this.loadTableDetails(); - } - } - - // Refresh when the component receives properties. - async componentDidUpdate() { - if (this.props.updateOnLoad) { - await this.loadTableDetails(this.props); - } - } - - // Leaving this render method alone during linting cleanup since it's - // used to discourage render without subclassing. - // eslint-disable-next-line react/require-render-return - render(): React.ReactElement { - throw new Error( - "DatabaseSummaryBase should never be instantiated directly. ", - ); - } -} - -export function databaseDetails(state: AdminUIState) { - return state.cachedData.databaseDetails; -} - -// Function which returns TableInfo objects for all tables in a database. This -// is not a selector, because it is not based only on the Redux state - it is -// also based on the tables in a single database. -// TODO(mrtracy): look into using a component like reselect-map if this proves -// to be expensive. My current intuition is that this will not be a bottleneck. -export function tableInfos(state: AdminUIState, dbName: string) { - const dbDetails = databaseDetails(state); - const tableNames = - dbDetails[dbName] && - dbDetails[dbName].data && - dbDetails[dbName].data.table_names; - if (!tableNames) { - return null; - } - const details = state.cachedData.tableDetails; - const stats = state.cachedData.tableStats; - return _.map(tableNames, (tableName) => { - const tblId = generateTableID(dbName, tableName); - const tblDetails = details[tblId] && details[tblId].data; - const tblStats = stats[tblId] && stats[tblId].data; - return new TableInfo(tableName, tblDetails, tblStats); - }); -} - -// Function which extracts the grants for a single database from redux state. -export function grants(state: AdminUIState, dbName: string) { - const dbDetails = databaseDetails(state); - return ( - dbDetails[dbName] && dbDetails[dbName].data && dbDetails[dbName].data.grants - ); -} diff --git a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.fixtures.ts b/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.fixtures.ts deleted file mode 100644 index e2aca9af608b..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.fixtures.ts +++ /dev/null @@ -1,376 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import { DatabaseSummaryProps } from "src/views/databases/containers/databaseSummary"; -import Long from "long"; -import { cockroach } from "src/js/protos"; -import ZoneConfigurationLevel = cockroach.server.serverpb.ZoneConfigurationLevel; -import moment from "moment"; -import { noop } from "lodash"; - -const mockedActions = { - setSort: () => {}, - refreshDatabaseDetails: () => {}, - refreshTableDetails: () => {}, - refreshTableStats: () => {}, -}; - -export const dbLoadingProps: any = { - ...mockedActions, - name: "defaultdb", - tableInfos: [], - dbResponse: { - inFlight: true, - }, - grants: [], -}; - -export const dbEmptyProps: any = { - ...mockedActions, - name: "defaultdb", - tableInfos: [], - dbResponse: { - inFlight: false, - }, - grants: [], -}; - -export const dbFullfilledProps: DatabaseSummaryProps = { - name: "system", - tableInfos: [ - { - name: "comments", - id: 24, - numColumns: 4, - numIndices: 1, - rangeCount: 1, - createStatement: - 'CREATE TABLE comments (\n\ttype INT8 NOT NULL,\n\tobject_id INT8 NOT NULL,\n\tsub_id INT8 NOT NULL,\n\tcomment STRING NOT NULL,\n\tCONSTRAINT "primary" PRIMARY KEY (type ASC, object_id ASC, sub_id ASC),\n\tFAMILY "primary" (type, object_id, sub_id),\n\tFAMILY fam_4_comment (comment)\n)', - configureZoneStatement: - "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", - grants: [ - { user: "admin", privileges: ["DELETE"] }, - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["INSERT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "admin", privileges: ["UPDATE"] }, - { user: "public", privileges: ["SELECT"] }, - { user: "root", privileges: ["DELETE"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["INSERT"] }, - { user: "root", privileges: ["SELECT"] }, - { user: "root", privileges: ["UPDATE"] }, - ], - numReplicas: 5, - mvccSize: { - last_update_nanos: Long.fromString("1568822946512088000"), - intent_age: Long.fromString("0"), - gc_bytes_age: Long.fromString("0"), - live_bytes: Long.fromString("0"), - live_count: Long.fromString("0"), - key_bytes: Long.fromString("0"), - key_count: Long.fromString("0"), - val_bytes: Long.fromString("0"), - val_count: Long.fromString("0"), - intent_bytes: Long.fromString("0"), - intent_count: Long.fromString("0"), - sys_bytes: Long.fromString("198"), - sys_count: Long.fromString("4"), - contains_estimates: Long.fromString("0"), - }, - physicalSize: 0, - }, - { - name: "descriptor", - id: 3, - numColumns: 2, - numIndices: 1, - rangeCount: 0, - createStatement: - 'CREATE TABLE descriptor (\n\tid INT8 NOT NULL,\n\tdescriptor BYTES NULL,\n\tCONSTRAINT "primary" PRIMARY KEY (id ASC),\n\tFAMILY "primary" (id),\n\tFAMILY fam_2_descriptor (descriptor)\n)', - configureZoneStatement: - "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", - grants: [ - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["SELECT"] }, - ], - numReplicas: 5, - mvccSize: { - last_update_nanos: Long.fromString("0"), - intent_age: Long.fromString("0"), - gc_bytes_age: Long.fromString("0"), - live_bytes: Long.fromString("0"), - live_count: Long.fromString("0"), - key_bytes: Long.fromString("0"), - key_count: Long.fromString("0"), - val_bytes: Long.fromString("0"), - val_count: Long.fromString("0"), - intent_bytes: Long.fromString("0"), - intent_count: Long.fromString("0"), - sys_bytes: Long.fromString("0"), - sys_count: Long.fromString("0"), - contains_estimates: Long.fromString("0"), - }, - physicalSize: 0, - }, - { - name: "eventlog", - id: 12, - numColumns: 6, - numIndices: 1, - rangeCount: 1, - createStatement: - 'CREATE TABLE eventlog (\n\t"timestamp" TIMESTAMP NOT NULL,\n\t"eventType" STRING NOT NULL,\n\t"targetID" INT8 NOT NULL,\n\t"reportingID" INT8 NOT NULL,\n\tinfo STRING NULL,\n\t"uniqueID" BYTES NOT NULL DEFAULT uuid_v4(),\n\tCONSTRAINT "primary" PRIMARY KEY ("timestamp" ASC, "uniqueID" ASC),\n\tFAMILY "primary" ("timestamp", "uniqueID"),\n\tFAMILY "fam_2_eventType" ("eventType"),\n\tFAMILY "fam_3_targetID" ("targetID"),\n\tFAMILY "fam_4_reportingID" ("reportingID"),\n\tFAMILY fam_5_info (info)\n)', - configureZoneStatement: - "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", - grants: [ - { user: "admin", privileges: ["DELETE"] }, - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["INSERT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "admin", privileges: ["UPDATE"] }, - { user: "root", privileges: ["DELETE"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["INSERT"] }, - { user: "root", privileges: ["SELECT"] }, - { user: "root", privileges: ["UPDATE"] }, - ], - numReplicas: 5, - mvccSize: { - last_update_nanos: Long.fromString("1590763503402517000"), - intent_age: Long.fromString("0"), - gc_bytes_age: Long.fromString("816043540"), - live_bytes: Long.fromString("13971"), - live_count: Long.fromString("90"), - key_bytes: Long.fromString("4531"), - key_count: Long.fromString("95"), - val_bytes: Long.fromString("10285"), - val_count: Long.fromString("100"), - intent_bytes: Long.fromString("0"), - intent_count: Long.fromString("0"), - sys_bytes: Long.fromString("208"), - sys_count: Long.fromString("4"), - contains_estimates: Long.fromString("0"), - }, - physicalSize: 0, - }, - { - name: "jobs", - id: 15, - numColumns: 5, - numIndices: 2, - rangeCount: 1, - createStatement: - "CREATE TABLE jobs (\n\tid INT8 NOT NULL DEFAULT unique_rowid(),\n\tstatus STRING NOT NULL,\n\tcreated TIMESTAMP NOT NULL DEFAULT now():::TIMESTAMP,\n\tpayload BYTES NOT NULL,\n\tprogress BYTES NULL,\n\tCONSTRAINT \"primary\" PRIMARY KEY (id ASC),\n\tINDEX jobs_status_created_idx (status ASC, created ASC),\n\tFAMILY fam_0_id_status_created_payload (id, status, created, payload),\n\tFAMILY progress (progress)\n);\nALTER TABLE system.public.jobs CONFIGURE ZONE USING\n\trange_min_bytes = 16777216,\n\trange_max_bytes = 67108864,\n\tgc.ttlseconds = 600,\n\tnum_replicas = 5,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", - configureZoneStatement: - "ALTER TABLE system.public.jobs CONFIGURE ZONE USING\n\trange_min_bytes = 16777216,\n\trange_max_bytes = 67108864,\n\tgc.ttlseconds = 600,\n\tnum_replicas = 5,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", - grants: [ - { user: "admin", privileges: ["DELETE"] }, - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["INSERT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "admin", privileges: ["UPDATE"] }, - { user: "root", privileges: ["DELETE"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["INSERT"] }, - { user: "root", privileges: ["SELECT"] }, - { user: "root", privileges: ["UPDATE"] }, - ], - numReplicas: 5, - mvccSize: { - last_update_nanos: Long.fromString("1589802430780575000"), - intent_age: Long.fromString("0"), - gc_bytes_age: Long.fromString("0"), - live_bytes: Long.fromString("0"), - live_count: Long.fromString("0"), - key_bytes: Long.fromString("0"), - key_count: Long.fromString("0"), - val_bytes: Long.fromString("0"), - val_count: Long.fromString("0"), - intent_bytes: Long.fromString("0"), - intent_count: Long.fromString("0"), - sys_bytes: Long.fromString("208"), - sys_count: Long.fromString("4"), - contains_estimates: Long.fromString("0"), - }, - physicalSize: 0, - }, - { - name: "lease", - id: 11, - numColumns: 4, - numIndices: 1, - rangeCount: 1, - createStatement: - 'CREATE TABLE lease (\n\t"descID" INT8 NOT NULL,\n\tversion INT8 NOT NULL,\n\t"nodeID" INT8 NOT NULL,\n\texpiration TIMESTAMP NOT NULL,\n\tCONSTRAINT "primary" PRIMARY KEY ("descID" ASC, version ASC, expiration ASC, "nodeID" ASC),\n\tFAMILY "primary" ("descID", version, "nodeID", expiration)\n)', - configureZoneStatement: - "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", - grants: [ - { user: "admin", privileges: ["DELETE"] }, - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["INSERT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "admin", privileges: ["UPDATE"] }, - { user: "root", privileges: ["DELETE"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["INSERT"] }, - { user: "root", privileges: ["SELECT"] }, - { user: "root", privileges: ["UPDATE"] }, - ], - numReplicas: 5, - mvccSize: { - last_update_nanos: Long.fromString("1591010093853736000"), - intent_age: Long.fromString("0"), - gc_bytes_age: Long.fromString("25775420"), - live_bytes: Long.fromString("70"), - live_count: Long.fromString("2"), - key_bytes: Long.fromString("5183"), - key_count: Long.fromString("124"), - val_bytes: Long.fromString("620"), - val_count: Long.fromString("246"), - intent_bytes: Long.fromString("0"), - intent_count: Long.fromString("0"), - sys_bytes: Long.fromString("208"), - sys_count: Long.fromString("4"), - contains_estimates: Long.fromString("0"), - }, - physicalSize: 0, - }, - { - name: "locations", - id: 21, - numColumns: 4, - numIndices: 1, - rangeCount: 1, - createStatement: - 'CREATE TABLE locations (\n\t"localityKey" STRING NOT NULL,\n\t"localityValue" STRING NOT NULL,\n\tlatitude DECIMAL(18,15) NOT NULL,\n\tlongitude DECIMAL(18,15) NOT NULL,\n\tCONSTRAINT "primary" PRIMARY KEY ("localityKey" ASC, "localityValue" ASC),\n\tFAMILY "fam_0_localityKey_localityValue_latitude_longitude" ("localityKey", "localityValue", latitude, longitude)\n)', - configureZoneStatement: - "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", - grants: [ - { user: "admin", privileges: ["DELETE"] }, - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["INSERT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "admin", privileges: ["UPDATE"] }, - { user: "root", privileges: ["DELETE"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["INSERT"] }, - { user: "root", privileges: ["SELECT"] }, - { user: "root", privileges: ["UPDATE"] }, - ], - numReplicas: 5, - mvccSize: { - last_update_nanos: Long.fromString("1568822946669858000"), - intent_age: Long.fromString("0"), - gc_bytes_age: Long.fromString("0"), - live_bytes: Long.fromString("326"), - live_count: Long.fromString("5"), - key_bytes: Long.fromString("187"), - key_count: Long.fromString("5"), - val_bytes: Long.fromString("139"), - val_count: Long.fromString("5"), - intent_bytes: Long.fromString("0"), - intent_count: Long.fromString("0"), - sys_bytes: Long.fromString("198"), - sys_count: Long.fromString("4"), - contains_estimates: Long.fromString("0"), - }, - physicalSize: 0, - }, - { - name: "namespace", - id: 30, - numColumns: 3, - numIndices: 1, - rangeCount: 1, - createStatement: - 'CREATE TABLE namespace (\n\t"parentID" INT8 NOT NULL,\n\tname STRING NOT NULL,\n\tid INT8 NULL,\n\tCONSTRAINT "primary" PRIMARY KEY ("parentID" ASC, name ASC),\n\tFAMILY "primary" ("parentID", name),\n\tFAMILY fam_3_id (id)\n)', - configureZoneStatement: - "ALTER RANGE default CONFIGURE ZONE USING\n\trange_min_bytes = 134217728,\n\trange_max_bytes = 536870912,\n\tgc.ttlseconds = 90000,\n\tnum_replicas = 3,\n\tconstraints = '[]',\n\tlease_preferences = '[]'", - grants: [ - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["SELECT"] }, - ], - numReplicas: 5, - mvccSize: { - last_update_nanos: Long.fromString("1587980617075991000"), - intent_age: Long.fromString("0"), - gc_bytes_age: Long.fromString("0"), - live_bytes: Long.fromString("1043"), - live_count: Long.fromString("27"), - key_bytes: Long.fromString("881"), - key_count: Long.fromString("27"), - val_bytes: Long.fromString("162"), - val_count: Long.fromString("27"), - intent_bytes: Long.fromString("0"), - intent_count: Long.fromString("0"), - sys_bytes: Long.fromString("250"), - sys_count: Long.fromString("4"), - contains_estimates: Long.fromString("0"), - }, - physicalSize: 0, - }, - ].map((ti) => ({ ...ti, detailsAndStatsLoaded: () => true })), - dbResponse: { - data: { - data: { - grants: [ - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["SELECT"] }, - ], - table_names: [ - "comments", - "descriptor", - "eventlog", - "jobs", - "lease", - "locations", - "namespace", - ], - descriptor_id: Long.fromString("1"), - zone_config: { - range_min_bytes: Long.fromString("16777216"), - range_max_bytes: Long.fromString("67108864"), - gc: { ttl_seconds: 90000 }, - num_replicas: 5, - inherited_constraints: false, - inherited_lease_preferences: false, - }, - zone_config_level: ZoneConfigurationLevel.DATABASE, - toJSON: () => ({}), - }, - inFlight: false, - valid: true, - requestedAt: moment("2020-06-01T11:15:23.907Z"), - setAt: moment("2020-06-01T11:15:24.323Z"), - lastError: null, - }, - }, - grants: [ - { user: "admin", privileges: ["GRANT"] }, - { user: "admin", privileges: ["SELECT"] }, - { user: "root", privileges: ["GRANT"] }, - { user: "root", privileges: ["SELECT"] }, - ].map((ti) => ({ ...ti, toJSON: () => ti })), - setSort: noop, - sortSetting: { - ascending: true, - sortKey: 1, - }, - refreshDatabaseDetails: noop as DatabaseSummaryProps["refreshDatabaseDetails"], - refreshTableDetails: noop as DatabaseSummaryProps["refreshTableDetails"], - refreshTableStats: noop as DatabaseSummaryProps["refreshTableStats"], -}; diff --git a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.spec.tsx b/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.spec.tsx deleted file mode 100644 index 1f5823033cc4..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.spec.tsx +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import React from "react"; -import _ from "lodash"; -import { assert } from "chai"; -import classNames from "classnames/bind"; -import { DatabaseSummaryTables } from "./"; -import { - dbLoadingProps, - dbEmptyProps, - dbFullfilledProps, -} from "./databaseTables.fixtures"; -import { connectedMount } from "src/test-utils"; - -import styles from "src/views/shared/components/sortabletable/sortabletable.module.styl"; -const cx = classNames.bind(styles); - -describe("", function () { - it("render loading state ", function () { - const wrapper = connectedMount(() => ( - - )); - assert.lengthOf(wrapper.find(`.${cx("table__loading")}`), 1); - }); - - it("render fulfilled state ", function () { - const wrapper = connectedMount(() => ( - - )); - assert.lengthOf(wrapper.find(`.${cx("sort-table__row--body")}`), 7); - }); - - it("render empty state ", function () { - const wrapper = connectedMount(() => ( - - )); - assert.lengthOf(wrapper.find(`.${cx("table__no-results")}`), 1); - }); -}); diff --git a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.stories.tsx b/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.stories.tsx deleted file mode 100644 index 54bbc34fab8b..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.stories.tsx +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. -import React from "react"; -import { storiesOf } from "@storybook/react"; -import { DatabaseSummaryTables } from "./"; -import { - dbEmptyProps, - dbLoadingProps, - dbFullfilledProps, -} from "./databaseTables.fixtures"; -import { withRouterDecorator } from "src/util/decorators"; - -storiesOf("DatabaseSummaryTables", module) - .addDecorator(withRouterDecorator) - .add("Loading state", () => ) - .add("Empty state", () => ) - .add("Default state", () => ); diff --git a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.styl b/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.styl deleted file mode 100644 index 059ab70c8126..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseTables/databaseTables.styl +++ /dev/null @@ -1,48 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -@require "~styl/base/palette.styl" - -.empty-state - display flex - justify-content center - align-items center - min-height 157px - flex-direction column - - &__line - margin 12px - -.table-icon - position relative - top 3px - right 2px - -.sort-table__cell a - color: $text-color; - &:hover - color: $link-color; - -.table-name a - display flex - align-items center - width 100% - height 100% - svg - margin-right 11px - &:hover - path - fill $colors--primary-blue-3 - -.database-summary-title - display flex - -.database-summary-load-button - margin-left 11px diff --git a/pkg/ui/src/views/databases/containers/databaseTables/index.tsx b/pkg/ui/src/views/databases/containers/databaseTables/index.tsx deleted file mode 100644 index 0d337c939b14..000000000000 --- a/pkg/ui/src/views/databases/containers/databaseTables/index.tsx +++ /dev/null @@ -1,213 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import _ from "lodash"; -import React from "react"; -import { connect } from "react-redux"; -import { Link } from "react-router-dom"; -import { - refreshDatabaseDetails, - refreshTableDetails, - refreshTableStats, -} from "src/redux/apiReducers"; -import { LocalSetting } from "src/redux/localsettings"; -import { AdminUIState } from "src/redux/state"; -import { Bytes } from "src/util/format"; -import { - databaseDetails, - DatabaseSummaryBase, - DatabaseSummaryExplicitData, - DatabaseSummaryProps, - grants, - tableInfos as selectTableInfos, -} from "src/views/databases/containers/databaseSummary"; -import { TableInfo } from "src/views/databases/data/tableInfo"; -import { SortSetting } from "src/views/shared/components/sortabletable"; -import { SortedTable } from "src/views/shared/components/sortedtable"; -import "./databaseTables.styl"; -import { DatabaseIcon } from "src/components/icon/databaseIcon"; -import Stack from "assets/stack.svg"; -import { SummaryCard } from "src/views/shared/components/summaryCard"; -import { SummaryHeadlineStat } from "src/views/shared/components/summaryBar"; -import TitleWithIcon from "../../components/titleWithIcon/titleWithIcon"; -import { ReplicatedSizeTooltip } from "src/views/databases/containers/databases/tooltips"; -import { Button } from "src/components"; - -const databaseTablesSortSetting = new LocalSetting( - "databases/sort_setting/tables", - (s) => s.localSettings, -); - -class DatabaseTableListSortedTable extends SortedTable {} - -// DatabaseSummaryTables displays a summary section describing the tables -// contained in a single database. -export class DatabaseSummaryTables extends DatabaseSummaryBase { - constructor(props: DatabaseSummaryProps) { - super(props); - - this.state = { - finishedLoadingTableData: - props.tableInfos && - props.tableInfos.every((ti) => ti.detailsAndStatsLoaded()), - }; - } - - totalSize() { - const tableInfos = this.props.tableInfos; - if (this.state.finishedLoadingTableData) { - return _.sumBy(tableInfos, (ti) => ti.physicalSize); - } else { - return null; - } - } - - totalRangeCount() { - const tableInfos = this.props.tableInfos; - if (this.state.finishedLoadingTableData) { - return _.sumBy(tableInfos, (ti) => ti.rangeCount); - } else { - return null; - } - } - - noDatabaseResults = () => ( - <> -

- - This database has no tables. -

- - ); - - render() { - const { tableInfos, dbResponse, sortSetting } = this.props; - const dbID = this.props.name; - const loading = dbResponse ? !!dbResponse.inFlight : true; - const numTables = (tableInfos && tableInfos.length) || 0; - return ( -
-
- - {this.state.finishedLoadingTableData || numTables === 0 ? null : ( - - )} -
-
-
- this.props.setSort(setting)} - firstCellBordered - columns={[ - { - title: "Table Name", - cell: (tableInfo) => { - return ( -
- - {tableInfo.name} - -
- ); - }, - sort: (tableInfo) => tableInfo.name, - className: "expand-link", // don't pad the td element to allow the link to expand - }, - { - title: ( - - {"Replicated Size"} - - ), - cell: (tableInfo) => - _.isUndefined(tableInfo.physicalSize) - ? "" - : Bytes(tableInfo.physicalSize), - sort: (tableInfo) => tableInfo.physicalSize, - }, - { - title: "Ranges", - cell: (tableInfo) => tableInfo.rangeCount, - sort: (tableInfo) => tableInfo.rangeCount, - }, - { - title: "# of Columns", - cell: (tableInfo) => tableInfo.numColumns, - sort: (tableInfo) => tableInfo.numColumns, - }, - { - title: "# of Indexes", - cell: (tableInfo) => tableInfo.numIndices, - sort: (tableInfo) => tableInfo.numIndices, - }, - ]} - loading={loading} - renderNoResult={loading ? undefined : this.noDatabaseResults()} - /> -
-
- - - - - -
-
-
- ); - } -} - -const mapStateToProps = ( - state: AdminUIState, - ownProps: DatabaseSummaryExplicitData, -) => ({ - // RootState contains declaration for whole state - tableInfos: selectTableInfos(state, ownProps.name), - sortSetting: databaseTablesSortSetting.selector(state), - dbResponse: databaseDetails(state)[ownProps.name], - grants: grants(state, ownProps.name), -}); - -const mapDispatchToProps = { - setSort: databaseTablesSortSetting.set, - refreshDatabaseDetails, - refreshTableDetails, - refreshTableStats, -}; - -// Connect the DatabaseSummaryTables class with redux store. -export default connect( - mapStateToProps, - mapDispatchToProps, -)(DatabaseSummaryTables as any); diff --git a/pkg/ui/src/views/databases/containers/databases/index.spec.ts b/pkg/ui/src/views/databases/containers/databases/index.spec.ts deleted file mode 100644 index 46b963f6aa2e..000000000000 --- a/pkg/ui/src/views/databases/containers/databases/index.spec.ts +++ /dev/null @@ -1,49 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import { assert } from "chai"; - -import * as protos from "src/js/protos"; -import { selectDatabasesByType } from "./"; - -describe("selectDatabasesByType", function () { - it("returns empty arrays if database data is missing", function () { - const state = { - cachedData: { - databases: { - inFlight: false, - valid: false, - }, - }, - }; - - assert.deepEqual(selectDatabasesByType(state), { user: [], system: [] }); - }); - - it("separates out the system databases", function () { - const userDatabases = ["foo", "bar", "baz"]; - const systemDatabases = ["defaultdb", "postgres", "system"]; - const state = { - cachedData: { - databases: { - inFlight: false, - valid: true, - data: protos.cockroach.server.serverpb.DatabasesResponse.fromObject({ - databases: systemDatabases.concat(userDatabases), - }), - }, - }, - }; - - const dbs = selectDatabasesByType(state); - - assert.deepEqual(dbs, { user: userDatabases, system: systemDatabases }); - }); -}); diff --git a/pkg/ui/src/views/databases/containers/databases/index.tsx b/pkg/ui/src/views/databases/containers/databases/index.tsx deleted file mode 100644 index d2c280c8480c..000000000000 --- a/pkg/ui/src/views/databases/containers/databases/index.tsx +++ /dev/null @@ -1,206 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import _ from "lodash"; -import React from "react"; -import { Helmet } from "react-helmet"; -import { connect } from "react-redux"; -import { createSelector } from "reselect"; -import { withRouter, RouteComponentProps } from "react-router-dom"; - -import Dropdown, { DropdownOption } from "src/views/shared/components/dropdown"; -import { - PageConfig, - PageConfigItem, -} from "src/views/shared/components/pageconfig"; - -import { AdminUIState } from "src/redux/state"; -import { refreshDatabases } from "src/redux/apiReducers"; -import { Pick } from "src/util/pick"; - -import DatabaseSummaryTables from "src/views/databases/containers/databaseTables"; -import DatabaseSummaryGrants from "src/views/databases/containers/databaseGrants"; -import NonTableSummary from "./nonTableSummary"; - -import "./databases.styl"; - -const databasePages = [ - { value: "tables", label: "Tables" }, - { value: "grants", label: "Grants" }, -]; - -// The system databases should sort after user databases. -const systemDatabases = ["defaultdb", "postgres", "system"]; - -interface DatabaseListNavProps { - selected: string; - onChange: (value: string) => void; -} -// DatabaseListNav displays the database page navigation bar. -class DatabaseListNav extends React.Component { - render() { - const { selected, onChange } = this.props; - return ( - - - { - onChange(value); - }} - /> - - - ); - } -} - -// DatabaseListData describes properties which should be passed to the -// DatabaseList container. -interface DatabaseListData { - // A list of databases for the user and the system. - databasesByType: { - user: string[]; - system: string[]; - }; -} - -// DatabaseListActions describes actions that can be dispatched by a -// DatabaseList component. -interface DatabaseListActions { - refreshDatabases: typeof refreshDatabases; -} - -type DatabaseListProps = DatabaseListData & - DatabaseListActions & - RouteComponentProps; - -// DatabaseTablesList displays the "Tables" sub-tab of the main database page. -class DatabaseTablesList extends React.Component { - componentDidMount() { - this.props.refreshDatabases(); - } - - handleOnNavigationListChange = (value: string) => { - this.props.history.push(`/databases/${value}`); - }; - - render() { - const { user, system } = this.props.databasesByType; - - return ( -
- -
-

Databases

-
- -
- {user.map((n) => ( - - ))} - {system.map((n) => ( - - ))} - -
-
- ); - } -} - -// DatabaseTablesList displays the "Grants" sub-tab of the main database page. -class DatabaseGrantsList extends React.Component { - componentDidMount() { - this.props.refreshDatabases(); - } - - handleOnNavigationListChange = (value: string) => { - this.props.history.push(`/databases/${value}`); - }; - - render() { - const { user, system } = this.props.databasesByType; - - return ( -
- -
-

Databases

-
- -
- {user.map((n) => ( - - ))} - {system.map((n) => ( - - ))} -
-
- ); - } -} - -type DatabasesState = Pick; - -// Base selectors to extract data from redux state. -function databaseNames(state: DatabasesState): string[] { - if ( - state.cachedData.databases.data && - state.cachedData.databases.data.databases - ) { - return state.cachedData.databases.data.databases; - } - return []; -} - -export const selectDatabasesByType = createSelector( - databaseNames, - (dbs: string[]) => { - const [user, system] = _.partition( - dbs, - (db) => systemDatabases.indexOf(db) === -1, - ); - return { user, system }; - }, -); - -const mapStateToProps = (state: AdminUIState) => ({ - // RootState contains declaration for whole state - databasesByType: selectDatabasesByType(state), -}); - -const mapDispatchToProps = { - refreshDatabases, -}; - -// Connect the DatabaseTablesList class with our redux store. -const databaseTablesListConnected = withRouter( - connect(mapStateToProps, mapDispatchToProps)(DatabaseTablesList), -); - -// Connect the DatabaseGrantsList class with our redux store. -const databaseGrantsListConnected = withRouter( - connect(mapStateToProps, mapDispatchToProps)(DatabaseGrantsList), -); - -export { - databaseTablesListConnected as DatabaseTablesList, - databaseGrantsListConnected as DatabaseGrantsList, -}; diff --git a/pkg/ui/src/views/databases/containers/databases/nonTableSummary.spec.tsx b/pkg/ui/src/views/databases/containers/databases/nonTableSummary.spec.tsx deleted file mode 100644 index 2c641fa29c77..000000000000 --- a/pkg/ui/src/views/databases/containers/databases/nonTableSummary.spec.tsx +++ /dev/null @@ -1,77 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import React from "react"; -import { shallow, mount } from "enzyme"; -import { noop } from "lodash"; -import { assert } from "chai"; -import Long from "long"; - -import "src/enzymeInit"; -import { NonTableSummary } from "./nonTableSummary"; -import { refreshNonTableStats } from "src/redux/apiReducers"; -import { cockroach } from "src/js/protos"; -import { Loading } from "@cockroachlabs/cluster-ui"; -import NonTableStatsResponse = cockroach.server.serverpb.NonTableStatsResponse; - -describe("NonTableSummary", () => { - describe("Loading data", () => { - it("successfully loads data", () => { - const tableStatsData = new NonTableStatsResponse({ - internal_use_stats: { - approximate_disk_bytes: Long.fromNumber(1), - missing_nodes: [], - node_count: Long.fromNumber(1), - range_count: Long.fromNumber(1), - replica_count: Long.fromNumber(1), - stats: null, - }, - time_series_stats: { - approximate_disk_bytes: Long.fromNumber(1), - missing_nodes: [], - node_count: Long.fromNumber(1), - range_count: Long.fromNumber(1), - replica_count: Long.fromNumber(1), - stats: null, - }, - }); - const wrapper = shallow( - , - ); - const loadingWrapper = wrapper.find(Loading).dive(); - assert.isTrue(loadingWrapper.find(".database-summary-table").exists()); - }); - - it("shows error message when failed request", () => { - const error = { - name: "Forbidden", - message: "Insufficient privileges to view this resource", - }; - - const wrapper = mount( - , - ); - - const loadingWrapper = wrapper.find(Loading); - assert.isTrue(loadingWrapper.exists()); - assert.isTrue(loadingWrapper.text().includes(error.message)); - }); - }); -}); diff --git a/pkg/ui/src/views/databases/containers/databases/nonTableSummary.tsx b/pkg/ui/src/views/databases/containers/databases/nonTableSummary.tsx deleted file mode 100644 index a99a511b5da9..000000000000 --- a/pkg/ui/src/views/databases/containers/databases/nonTableSummary.tsx +++ /dev/null @@ -1,137 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import React from "react"; -import { connect } from "react-redux"; -import * as protos from "src/js/protos"; -import { refreshNonTableStats } from "src/redux/apiReducers"; -import { AdminUIState } from "src/redux/state"; -import { FixLong } from "src/util/fixLong"; -import { Bytes } from "src/util/format"; -import { Loading } from "@cockroachlabs/cluster-ui"; -import { CachedDataReducerState } from "src/redux/cachedDataReducer"; -import { NonTableStatsResponseMessage } from "src/util/api"; -import { TimeSeriesTooltip } from "src/views/databases/containers/databases/tooltips"; -import "src/views/shared/components/sortabletable/sortabletable.styl"; - -interface TimeSeriesSummaryProps { - nonTableStats: protos.cockroach.server.serverpb.NonTableStatsResponse; - // Must be connected to react-redux in order to auto-refresh time series - // information. - nonTableStatsValid: boolean; - refreshNonTableStats: typeof refreshNonTableStats; - lastError: CachedDataReducerState["lastError"]; -} - -// NonTableSummary displays a summary section describing the current data -// usage of the time series system. -export class NonTableSummary extends React.Component { - componentDidMount() { - // Refresh nodes status query when mounting. - this.props.refreshNonTableStats(); - } - - componentDidUpdate() { - // Refresh nodes status query when props are received; this will immediately - // trigger a new request if previous results are invalidated. - this.props.refreshNonTableStats(); - } - - renderTable = () => { - return ( -
- - - - - - - - - - - - - - - - - - - - -
Data TypeSizeRanges
- Time Series - - {Bytes( - FixLong( - this.props.nonTableStats.time_series_stats - .approximate_disk_bytes, - ).toNumber(), - )} - - {FixLong( - this.props.nonTableStats.time_series_stats.range_count, - ).toNumber()} -
Internal Use - {Bytes( - FixLong( - this.props.nonTableStats.internal_use_stats - .approximate_disk_bytes, - ).toNumber(), - )} - - {FixLong( - this.props.nonTableStats.internal_use_stats.range_count, - ).toNumber()} -
-
- ); - }; - - render() { - return ( -
-
-

Non-Table Cluster Data

-
-
-
- -
-
-
-
- ); - } -} - -// Base selectors to extract data from redux state. -const nonTableStatsData = (state: AdminUIState) => - state.cachedData.nonTableStats; - -const mapStateToProps = (state: AdminUIState) => { - const ntStats = nonTableStatsData(state); - return { - nonTableStats: ntStats && ntStats.data, - nonTableStatsValid: ntStats && ntStats.valid, - lastError: ntStats && ntStats.lastError, - }; -}; - -const mapDispatchToProps = { - refreshNonTableStats, -}; - -export default connect(mapStateToProps, mapDispatchToProps)(NonTableSummary); diff --git a/pkg/ui/src/views/databases/containers/databases/tooltips.tsx b/pkg/ui/src/views/databases/containers/databases/tooltips.tsx deleted file mode 100644 index 67e2de80ab1a..000000000000 --- a/pkg/ui/src/views/databases/containers/databases/tooltips.tsx +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import React from "react"; -import { Tooltip } from "src/components"; -import { TooltipProps } from "src/components/tooltip/tooltip"; - -export const TimeSeriesTooltip: React.FC = (props) => ( - -

Total disk size of Admin UI metrics.

-
- } - > - {props.children} - -); - -export const ReplicatedSizeTooltip: React.FC< - TooltipProps & { tableName: string } -> = (props) => ( - -

- Approximate disk size of all replicas of table{" "} - {props.tableName || ""} on the cluster. -

- - } - > - {props.children} - -); diff --git a/pkg/ui/src/views/databases/containers/tableDetails/index.tsx b/pkg/ui/src/views/databases/containers/tableDetails/index.tsx deleted file mode 100644 index 6abb5dd20c06..000000000000 --- a/pkg/ui/src/views/databases/containers/tableDetails/index.tsx +++ /dev/null @@ -1,241 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import { Col, Row, Tabs } from "antd"; -import { SummaryCard } from "src/views/shared/components/summaryCard"; -import React from "react"; -import { Helmet } from "react-helmet"; -import { connect } from "react-redux"; -import { RouteComponentProps, withRouter } from "react-router-dom"; -import * as protos from "src/js/protos"; -import { - generateTableID, - refreshTableDetails, - refreshTableStats, - refreshDatabaseDetails, -} from "src/redux/apiReducers"; -import { LocalSetting } from "src/redux/localsettings"; -import { AdminUIState } from "src/redux/state"; -import { databaseNameAttr, tableNameAttr } from "src/util/constants"; -import { Bytes } from "src/util/format"; -import { TableInfo } from "src/views/databases/data/tableInfo"; -import { SortSetting } from "src/views/shared/components/sortabletable"; -import { SortedTable } from "src/views/shared/components/sortedtable"; -const { TabPane } = Tabs; -import { getMatchParamByName } from "src/util/query"; -import { Button } from "@cockroachlabs/cluster-ui"; -import { ArrowLeft } from "@cockroachlabs/icons"; -import SqlBox from "src/views/shared/components/sql/box"; - -class GrantsSortedTable extends SortedTable {} - -const databaseTableGrantsSortSetting = new LocalSetting< - AdminUIState, - SortSetting ->("tableDetails/sort_setting/grants", (s) => s.localSettings); - -/** - * TableMainData are the data properties which should be passed to the TableMain - * container. - */ -interface TableMainData { - tableInfo: TableInfo; - grantsSortSetting: SortSetting; -} - -/** - * TableMainActions are the action dispatchers which should be passed to the - * TableMain container. - */ -interface TableMainActions { - // Refresh the table data - refreshTableDetails: typeof refreshTableDetails; - refreshTableStats: typeof refreshTableStats; - refreshDatabaseDetails: typeof refreshDatabaseDetails; - setSort: typeof databaseTableGrantsSortSetting.set; -} - -/** - * TableMainProps is the type of the props object that must be passed to - * TableMain component. - */ -type TableMainProps = TableMainData & TableMainActions & RouteComponentProps; - -/** - * TableMain renders the main content of the databases page, which is primarily a - * data table of all databases. - */ -export class TableMain extends React.Component { - componentDidMount() { - const database = getMatchParamByName(this.props.match, databaseNameAttr); - const table = getMatchParamByName(this.props.match, tableNameAttr); - this.props.refreshDatabaseDetails( - new protos.cockroach.server.serverpb.DatabaseDetailsRequest({ - database: getMatchParamByName(this.props.match, databaseNameAttr), - }), - ); - this.props.refreshTableDetails( - new protos.cockroach.server.serverpb.TableDetailsRequest({ - database, - table, - }), - ); - this.props.refreshTableStats( - new protos.cockroach.server.serverpb.TableStatsRequest({ - database, - table, - }), - ); - } - - prevPage = () => this.props.history.goBack(); - - render() { - const { tableInfo, grantsSortSetting, match } = this.props; - const database = getMatchParamByName(match, databaseNameAttr); - const table = getMatchParamByName(match, tableNameAttr); - - const title = `${database}.${table}`; - if (tableInfo) { - return ( -
- -
- -
-

{title}

-
-
-
- - - -
- - - - - - -
-

- {Bytes(tableInfo.physicalSize)} -

-

- Size -

-
- - -
-

- {tableInfo.numReplicas} -

-

- Replicas -

-
- - -
-

- {tableInfo.rangeCount} -

-

- Ranges -

-
- - - - - - - - - - this.props.setSort(setting) - } - columns={[ - { - title: "User", - cell: (grants) => grants.user, - sort: (grants) => grants.user, - }, - { - title: "Grants", - cell: (grants) => grants.privileges.join(", "), - sort: (grants) => grants.privileges.join(", "), - }, - ]} - /> - - - - - - ); - } - return
No results.
; - } -} - -/****************************** - * SELECTORS - */ - -export function selectTableInfo( - state: AdminUIState, - props: RouteComponentProps, -): TableInfo { - const db = getMatchParamByName(props.match, databaseNameAttr); - const table = getMatchParamByName(props.match, tableNameAttr); - const details = state.cachedData.tableDetails[generateTableID(db, table)]; - const stats = state.cachedData.tableStats[generateTableID(db, table)]; - return new TableInfo(table, details && details.data, stats && stats.data); -} - -const mapStateToProps = ( - state: AdminUIState, - ownProps: RouteComponentProps, -) => { - return { - tableInfo: selectTableInfo(state, ownProps), - grantsSortSetting: databaseTableGrantsSortSetting.selector(state), - }; -}; - -const mapDispatchToProps = { - setSort: databaseTableGrantsSortSetting.set, - refreshTableDetails, - refreshTableStats, - refreshDatabaseDetails, -}; - -// Connect the TableMain class with our redux store. -const tableMainConnected = withRouter( - connect(mapStateToProps, mapDispatchToProps)(TableMain), -); - -export default tableMainConnected; diff --git a/pkg/ui/src/views/databases/data/tableInfo.tsx b/pkg/ui/src/views/databases/data/tableInfo.tsx deleted file mode 100644 index 7dc11cd12e5a..000000000000 --- a/pkg/ui/src/views/databases/data/tableInfo.tsx +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import * as protos from "src/js/protos"; -import { FixLong } from "src/util/fixLong"; - -import _ from "lodash"; - -type TableDetailsResponse = protos.cockroach.server.serverpb.TableDetailsResponse; -type TableStatsResponse = protos.cockroach.server.serverpb.TableStatsResponse; - -// TableInfo is a supporting data structure which combines data about a single -// table that was obtained from multiple backend sources. -export class TableInfo { - public name: string; - public id: number; - public numColumns: number; - public numIndices: number; - public physicalSize: number; - public mvccSize: protos.cockroach.storage.enginepb.IMVCCStats; - public rangeCount: number; - public createStatement: string; - public configureZoneStatement: string; - public grants: protos.cockroach.server.serverpb.TableDetailsResponse.IGrant[]; - public numReplicas: number; - constructor( - name: string, - details: TableDetailsResponse, - stats: TableStatsResponse, - ) { - this.name = name; - this.id = details && details.descriptor_id.toNumber(); - this.numColumns = details && details.columns.length; - this.numIndices = - details && _.uniqBy(details.indexes, (idx) => idx.name).length; - this.rangeCount = - stats && stats.range_count && stats.range_count.toNumber(); - this.createStatement = details && details.create_table_statement; - this.configureZoneStatement = details && details.configure_zone_statement; - this.grants = details && details.grants; - this.numReplicas = - details && details.zone_config && details.zone_config.num_replicas; - if (stats) { - this.mvccSize = stats.stats; - this.physicalSize = FixLong(stats.approximate_disk_bytes).toNumber(); - } - } - - public detailsAndStatsLoaded(): boolean { - return this.id !== undefined && this.physicalSize !== undefined; - } -} diff --git a/pkg/ui/src/views/databases/databaseDetailsPage/index.ts b/pkg/ui/src/views/databases/databaseDetailsPage/index.ts new file mode 100644 index 000000000000..bc5ebeeac0de --- /dev/null +++ b/pkg/ui/src/views/databases/databaseDetailsPage/index.ts @@ -0,0 +1,21 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { connect } from "react-redux"; +import { withRouter } from "react-router-dom"; +import { DatabaseDetailsPage } from "@cockroachlabs/cluster-ui"; + +import { mapStateToProps, mapDispatchToProps } from "./redux"; + +const connected = withRouter( + connect(mapStateToProps, mapDispatchToProps)(DatabaseDetailsPage), +); + +export { connected as DatabaseDetailsPage }; diff --git a/pkg/ui/src/views/databases/databaseDetailsPage/redux.spec.ts b/pkg/ui/src/views/databases/databaseDetailsPage/redux.spec.ts new file mode 100644 index 000000000000..54cb8c8f6c8c --- /dev/null +++ b/pkg/ui/src/views/databases/databaseDetailsPage/redux.spec.ts @@ -0,0 +1,329 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import assert from "assert"; +import { createMemoryHistory } from "history"; +import _ from "lodash"; +import Long from "long"; +import { RouteComponentProps } from "react-router-dom"; +import { bindActionCreators, Store } from "redux"; +import { + DatabaseDetailsPageActions, + DatabaseDetailsPageData, + DatabaseDetailsPageDataTableDetails, + DatabaseDetailsPageDataTableStats, +} from "@cockroachlabs/cluster-ui"; + +import { AdminUIState, createAdminUIStore } from "src/redux/state"; +import { databaseNameAttr } from "src/util/constants"; +import * as fakeApi from "src/util/fakeApi"; +import { mapStateToProps, mapDispatchToProps } from "./redux"; + +function fakeRouteComponentProps( + key: string, + value: string, +): RouteComponentProps { + return { + history: createMemoryHistory(), + location: { + pathname: "", + search: "", + state: {}, + hash: "", + }, + match: { + params: { + [key]: value, + }, + isExact: true, + path: "", + url: "", + }, + }; +} + +class TestDriver { + private readonly actions: DatabaseDetailsPageActions; + private readonly properties: () => DatabaseDetailsPageData; + + constructor(store: Store, private readonly database: string) { + this.actions = bindActionCreators( + mapDispatchToProps, + store.dispatch.bind(store), + ); + this.properties = () => + mapStateToProps( + store.getState(), + fakeRouteComponentProps(databaseNameAttr, database), + ); + } + + assertProperties(expected: DatabaseDetailsPageData) { + assert.deepEqual(this.properties(), expected); + } + + assertTableDetails( + name: string, + expected: DatabaseDetailsPageDataTableDetails, + ) { + assert.deepEqual(this.findTable(name).details, expected); + } + + assertTableRoles(name: string, expected: string[]) { + assert.deepEqual(this.findTable(name).details.roles, expected); + } + + assertTableGrants(name: string, expected: string[]) { + assert.deepEqual(this.findTable(name).details.grants, expected); + } + + assertTableStats(name: string, expected: DatabaseDetailsPageDataTableStats) { + assert.deepEqual(this.findTable(name).stats, expected); + } + + async refreshDatabaseDetails() { + return this.actions.refreshDatabaseDetails(this.database); + } + + async refreshTableDetails(table: string) { + return this.actions.refreshTableDetails(this.database, table); + } + + async refreshTableStats(table: string) { + return this.actions.refreshTableStats(this.database, table); + } + + private findTable(name: string) { + return _.find(this.properties().tables, { name }); + } +} + +describe("Database Details Page", function () { + let driver: TestDriver; + + beforeEach(function () { + driver = new TestDriver( + createAdminUIStore(createMemoryHistory()), + "things", + ); + }); + + afterEach(function () { + fakeApi.restore(); + }); + + it("starts in a pre-loading state", function () { + driver.assertProperties({ + loading: false, + loaded: false, + name: "things", + tables: [], + }); + }); + + it("makes a row for each table", async function () { + fakeApi.stubDatabaseDetails("things", { + table_names: ["foo", "bar"], + }); + + await driver.refreshDatabaseDetails(); + + driver.assertProperties({ + loading: false, + loaded: true, + name: "things", + tables: [ + { + name: "foo", + details: { + loading: false, + loaded: false, + columnCount: 0, + indexCount: 0, + userCount: 0, + roles: [], + grants: [], + }, + stats: { + loading: false, + loaded: false, + replicationSizeInBytes: 0, + rangeCount: 0, + }, + }, + { + name: "bar", + details: { + loading: false, + loaded: false, + columnCount: 0, + indexCount: 0, + userCount: 0, + roles: [], + grants: [], + }, + stats: { + loading: false, + loaded: false, + replicationSizeInBytes: 0, + rangeCount: 0, + }, + }, + ], + }); + }); + + it("loads table details", async function () { + fakeApi.stubDatabaseDetails("things", { + table_names: ["foo", "bar"], + }); + + fakeApi.stubTableDetails("things", "foo", { + grants: [ + { user: "admin", privileges: ["CREATE"] }, + { user: "public", privileges: ["SELECT"] }, + ], + // The actual contents below don't matter to us; we just count them. + columns: [{}, {}, {}, {}, {}], + indexes: [{}, {}, {}], + }); + + fakeApi.stubTableDetails("things", "bar", { + grants: [ + { user: "root", privileges: ["ALL"] }, + { user: "app", privileges: ["INSERT"] }, + { user: "data", privileges: ["SELECT"] }, + ], + // The actual contents below don't matter to us; we just count them. + columns: [{}, {}, {}, {}], + indexes: [{}, {}], + }); + + await driver.refreshDatabaseDetails(); + await driver.refreshTableDetails("foo"); + await driver.refreshTableDetails("bar"); + + driver.assertTableDetails("foo", { + loading: false, + loaded: true, + columnCount: 5, + indexCount: 3, + userCount: 2, + roles: ["admin", "public"], + grants: ["CREATE", "SELECT"], + }); + + driver.assertTableDetails("bar", { + loading: false, + loaded: true, + columnCount: 4, + indexCount: 2, + userCount: 3, + roles: ["root", "app", "data"], + grants: ["ALL", "SELECT", "INSERT"], + }); + }); + + it("sorts roles meaningfully", async function () { + fakeApi.stubDatabaseDetails("things", { + table_names: ["foo"], + }); + + fakeApi.stubTableDetails("things", "foo", { + grants: [ + { user: "bzuckercorn", privileges: ["ALL"] }, + { user: "bloblaw", privileges: ["ALL"] }, + { user: "jwweatherman", privileges: ["ALL"] }, + { user: "admin", privileges: ["ALL"] }, + { user: "public", privileges: ["ALL"] }, + { user: "root", privileges: ["ALL"] }, + ], + }); + + await driver.refreshDatabaseDetails(); + await driver.refreshTableDetails("foo"); + + driver.assertTableRoles("foo", [ + "root", + "admin", + "public", + "bloblaw", + "bzuckercorn", + "jwweatherman", + ]); + }); + + it("sorts grants meaningfully", async function () { + fakeApi.stubDatabaseDetails("things", { + table_names: ["foo"], + }); + + fakeApi.stubTableDetails("things", "foo", { + grants: [ + { + user: "admin", + privileges: ["ALL", "CREATE", "DELETE", "DROP", "GRANT"], + }, + { + user: "public", + privileges: ["DROP", "GRANT", "INSERT", "SELECT", "UPDATE"], + }, + ], + }); + + await driver.refreshDatabaseDetails(); + await driver.refreshTableDetails("foo"); + + driver.assertTableGrants("foo", [ + "ALL", + "CREATE", + "DROP", + "GRANT", + "SELECT", + "INSERT", + "UPDATE", + "DELETE", + ]); + }); + + it("loads table stats", async function () { + fakeApi.stubDatabaseDetails("things", { + table_names: ["foo", "bar"], + }); + + fakeApi.stubTableStats("things", "foo", { + range_count: new Long(4200), + approximate_disk_bytes: new Long(44040192), + }); + + fakeApi.stubTableStats("things", "bar", { + range_count: new Long(1023), + approximate_disk_bytes: new Long(8675309), + }); + + await driver.refreshDatabaseDetails(); + await driver.refreshTableStats("foo"); + await driver.refreshTableStats("bar"); + + driver.assertTableStats("foo", { + loading: false, + loaded: true, + replicationSizeInBytes: 44040192, + rangeCount: 4200, + }); + + driver.assertTableStats("bar", { + loading: false, + loaded: true, + replicationSizeInBytes: 8675309, + rangeCount: 1023, + }); + }); +}); diff --git a/pkg/ui/src/views/databases/databaseDetailsPage/redux.ts b/pkg/ui/src/views/databases/databaseDetailsPage/redux.ts new file mode 100644 index 000000000000..9058ddecc7a1 --- /dev/null +++ b/pkg/ui/src/views/databases/databaseDetailsPage/redux.ts @@ -0,0 +1,135 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { RouteComponentProps } from "react-router"; +import { createSelector } from "reselect"; +import _ from "lodash"; +import { DatabaseDetailsPageData } from "@cockroachlabs/cluster-ui"; + +import { cockroach } from "src/js/protos"; +import { + generateTableID, + refreshDatabaseDetails, + refreshTableDetails, + refreshTableStats, +} from "src/redux/apiReducers"; +import { AdminUIState } from "src/redux/state"; +import { databaseNameAttr } from "src/util/constants"; +import { FixLong } from "src/util/fixLong"; +import { getMatchParamByName } from "src/util/query"; + +const { + DatabaseDetailsRequest, + TableDetailsRequest, + TableStatsRequest, +} = cockroach.server.serverpb; + +function normalizeRoles(raw: string[]): string[] { + const rolePrecedence: Record = { + root: 1, + admin: 2, + public: 3, + }; + + // Once we have an alphabetized list of roles, we sort it again, promoting + // root, admin, and public to the head of the list. (We rely on _.sortBy to + // be stable.) + const alphabetizedRoles = _.sortBy(_.uniq(_.filter(raw))); + + return _.sortBy(alphabetizedRoles, (role) => rolePrecedence[role] || 100); +} + +function normalizePrivileges(raw: string[]): string[] { + const privilegePrecedence: Record = { + ALL: 1, + CREATE: 2, + DROP: 3, + GRANT: 4, + SELECT: 5, + INSERT: 6, + UPDATE: 7, + DELETE: 8, + }; + + return _.sortBy( + _.uniq(_.map(_.filter(raw), _.toUpper)), + (privilege) => privilegePrecedence[privilege] || 100, + ); +} + +export const mapStateToProps = createSelector( + (_state: AdminUIState, props: RouteComponentProps): string => + getMatchParamByName(props.match, databaseNameAttr), + + (state) => state.cachedData.databaseDetails, + (state) => state.cachedData.tableDetails, + (state) => state.cachedData.tableStats, + + ( + database, + databaseDetails, + tableDetails, + tableStats, + ): DatabaseDetailsPageData => { + return { + loading: !!databaseDetails[database]?.inFlight, + loaded: !!databaseDetails[database]?.valid, + name: database, + tables: _.map(databaseDetails[database]?.data?.table_names, (table) => { + const tableId = generateTableID(database, table); + + const details = tableDetails[tableId]; + const stats = tableStats[tableId]; + + const roles = normalizeRoles(_.map(details?.data?.grants, "user")); + const grants = normalizePrivileges( + _.flatMap(details?.data?.grants, "privileges"), + ); + + return { + name: table, + details: { + loading: !!details?.inFlight, + loaded: !!details?.valid, + columnCount: details?.data?.columns?.length || 0, + indexCount: details?.data?.indexes?.length || 0, + userCount: roles.length, + roles: roles, + grants: grants, + }, + stats: { + loading: !!stats?.inFlight, + loaded: !!stats?.valid, + replicationSizeInBytes: FixLong( + stats?.data?.approximate_disk_bytes || 0, + ).toNumber(), + rangeCount: FixLong(stats?.data?.range_count || 0).toNumber(), + }, + }; + }), + }; + }, +); + +export const mapDispatchToProps = { + refreshDatabaseDetails: (database: string) => { + return refreshDatabaseDetails( + new DatabaseDetailsRequest({ database, include_stats: true }), + ); + }, + + refreshTableDetails: (database: string, table: string) => { + return refreshTableDetails(new TableDetailsRequest({ database, table })); + }, + + refreshTableStats: (database: string, table: string) => { + return refreshTableStats(new TableStatsRequest({ database, table })); + }, +}; diff --git a/pkg/ui/src/views/databases/databaseTablePage/index.ts b/pkg/ui/src/views/databases/databaseTablePage/index.ts new file mode 100644 index 000000000000..80cc6f13debe --- /dev/null +++ b/pkg/ui/src/views/databases/databaseTablePage/index.ts @@ -0,0 +1,21 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { connect } from "react-redux"; +import { withRouter } from "react-router-dom"; +import { DatabaseTablePage } from "@cockroachlabs/cluster-ui"; + +import { mapStateToProps, mapDispatchToProps } from "./redux"; + +const connected = withRouter( + connect(mapStateToProps, mapDispatchToProps)(DatabaseTablePage), +); + +export { connected as DatabaseTablePage }; diff --git a/pkg/ui/src/views/databases/databaseTablePage/redux.spec.ts b/pkg/ui/src/views/databases/databaseTablePage/redux.spec.ts new file mode 100644 index 000000000000..540dbaf340aa --- /dev/null +++ b/pkg/ui/src/views/databases/databaseTablePage/redux.spec.ts @@ -0,0 +1,185 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import assert from "assert"; +import { createMemoryHistory } from "history"; +import _ from "lodash"; +import Long from "long"; +import { RouteComponentProps } from "react-router-dom"; +import { bindActionCreators, Store } from "redux"; +import { + DatabaseTablePageActions, + DatabaseTablePageData, + DatabaseTablePageDataDetails, + DatabaseTablePageDataStats, +} from "@cockroachlabs/cluster-ui"; + +import { AdminUIState, createAdminUIStore } from "src/redux/state"; +import { databaseNameAttr, tableNameAttr } from "src/util/constants"; +import * as fakeApi from "src/util/fakeApi"; +import { mapStateToProps, mapDispatchToProps } from "./redux"; + +function fakeRouteComponentProps( + k1: string, + v1: string, + k2: string, + v2: string, +): RouteComponentProps { + return { + history: createMemoryHistory(), + location: { + pathname: "", + search: "", + state: {}, + hash: "", + }, + match: { + params: { + [k1]: v1, + [k2]: v2, + }, + isExact: true, + path: "", + url: "", + }, + }; +} + +class TestDriver { + private readonly actions: DatabaseTablePageActions; + private readonly properties: () => DatabaseTablePageData; + + constructor( + store: Store, + private readonly database: string, + private readonly table: string, + ) { + this.actions = bindActionCreators( + mapDispatchToProps, + store.dispatch.bind(store), + ); + this.properties = () => + mapStateToProps( + store.getState(), + fakeRouteComponentProps( + databaseNameAttr, + database, + tableNameAttr, + table, + ), + ); + } + + assertProperties(expected: DatabaseTablePageData) { + assert.deepEqual(this.properties(), expected); + } + + assertTableDetails(expected: DatabaseTablePageDataDetails) { + assert.deepEqual(this.properties().details, expected); + } + + assertTableStats(expected: DatabaseTablePageDataStats) { + assert.deepEqual(this.properties().stats, expected); + } + + async refreshTableDetails() { + return this.actions.refreshTableDetails(this.database, this.table); + } + + async refreshTableStats() { + return this.actions.refreshTableStats(this.database, this.table); + } +} + +describe("Database Table Page", function () { + let driver: TestDriver; + + beforeEach(function () { + driver = new TestDriver( + createAdminUIStore(createMemoryHistory()), + "DATABASE", + "TABLE", + ); + }); + + afterEach(function () { + fakeApi.restore(); + }); + + it("starts in a pre-loading state", function () { + driver.assertProperties({ + databaseName: "DATABASE", + name: "TABLE", + details: { + loading: false, + loaded: false, + createStatement: "", + replicaCount: 0, + indexNames: [], + grants: [], + }, + stats: { + loading: false, + loaded: false, + sizeInBytes: 0, + rangeCount: 0, + }, + }); + }); + + it("loads table details", async function () { + fakeApi.stubTableDetails("DATABASE", "TABLE", { + grants: [ + { user: "admin", privileges: ["CREATE", "DROP"] }, + { user: "public", privileges: ["SELECT"] }, + ], + indexes: [ + { name: "primary" }, + { name: "another_index", seq: new Long(1) }, + { name: "another_index", seq: new Long(2) }, + ], + create_table_statement: "CREATE TABLE foo", + zone_config: { + num_replicas: 5, + }, + }); + + await driver.refreshTableDetails(); + + driver.assertTableDetails({ + loading: false, + loaded: true, + createStatement: "CREATE TABLE foo", + replicaCount: 5, + indexNames: ["primary", "another_index"], + grants: [ + { user: "admin", privilege: "CREATE" }, + { user: "admin", privilege: "DROP" }, + { user: "public", privilege: "SELECT" }, + ], + }); + }); + + it("loads table stats", async function () { + fakeApi.stubTableStats("DATABASE", "TABLE", { + range_count: new Long(4200), + approximate_disk_bytes: new Long(44040192), + }); + + await driver.refreshTableStats(); + + driver.assertTableStats({ + loading: false, + loaded: true, + sizeInBytes: 44040192, + rangeCount: 4200, + }); + }); +}); diff --git a/pkg/ui/src/views/databases/databaseTablePage/redux.ts b/pkg/ui/src/views/databases/databaseTablePage/redux.ts new file mode 100644 index 000000000000..7e8667f71b9a --- /dev/null +++ b/pkg/ui/src/views/databases/databaseTablePage/redux.ts @@ -0,0 +1,80 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { RouteComponentProps } from "react-router"; +import { createSelector } from "reselect"; +import _ from "lodash"; +import { DatabaseTablePageData } from "@cockroachlabs/cluster-ui"; + +import { cockroach } from "src/js/protos"; +import { + generateTableID, + refreshTableDetails, + refreshTableStats, +} from "src/redux/apiReducers"; +import { AdminUIState } from "src/redux/state"; +import { databaseNameAttr, tableNameAttr } from "src/util/constants"; +import { FixLong } from "src/util/fixLong"; +import { getMatchParamByName } from "src/util/query"; + +const { TableDetailsRequest, TableStatsRequest } = cockroach.server.serverpb; + +export const mapStateToProps = createSelector( + (_state: AdminUIState, props: RouteComponentProps): string => + getMatchParamByName(props.match, databaseNameAttr), + (_state: AdminUIState, props: RouteComponentProps): string => + getMatchParamByName(props.match, tableNameAttr), + + (state) => state.cachedData.tableDetails, + (state) => state.cachedData.tableStats, + + (database, table, tableDetails, tableStats): DatabaseTablePageData => { + const details = tableDetails[generateTableID(database, table)]; + const stats = tableStats[generateTableID(database, table)]; + const grants = _.flatMap(details?.data?.grants, (grant) => + _.map(grant.privileges, (privilege) => { + return { user: grant.user, privilege }; + }), + ); + + return { + databaseName: database, + name: table, + details: { + loading: !!details?.inFlight, + loaded: !!details?.valid, + createStatement: details?.data?.create_table_statement || "", + replicaCount: details?.data?.zone_config?.num_replicas || 0, + indexNames: _.uniq( + _.map(details?.data?.indexes, (index) => index.name), + ), + grants: grants, + }, + stats: { + loading: !!stats?.inFlight, + loaded: !!stats?.valid, + sizeInBytes: FixLong( + stats?.data?.approximate_disk_bytes || 0, + ).toNumber(), + rangeCount: FixLong(stats?.data?.range_count || 0).toNumber(), + }, + }; + }, +); + +export const mapDispatchToProps = { + refreshTableDetails: (database: string, table: string) => { + return refreshTableDetails(new TableDetailsRequest({ database, table })); + }, + + refreshTableStats: (database: string, table: string) => { + return refreshTableStats(new TableStatsRequest({ database, table })); + }, +}; diff --git a/pkg/ui/src/views/databases/databasesPage/index.ts b/pkg/ui/src/views/databases/databasesPage/index.ts new file mode 100644 index 000000000000..e1c30f42e6c6 --- /dev/null +++ b/pkg/ui/src/views/databases/databasesPage/index.ts @@ -0,0 +1,21 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { connect } from "react-redux"; +import { withRouter } from "react-router-dom"; +import { DatabasesPage } from "@cockroachlabs/cluster-ui"; + +import { mapStateToProps, mapDispatchToProps } from "./redux"; + +const connected = withRouter( + connect(mapStateToProps, mapDispatchToProps)(DatabasesPage), +); + +export { connected as DatabasesPage }; diff --git a/pkg/ui/src/views/databases/databasesPage/redux.spec.ts b/pkg/ui/src/views/databases/databasesPage/redux.spec.ts new file mode 100644 index 000000000000..639a60c6e2b0 --- /dev/null +++ b/pkg/ui/src/views/databases/databasesPage/redux.spec.ts @@ -0,0 +1,321 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import assert from "assert"; +import { createMemoryHistory } from "history"; +import _ from "lodash"; +import Long from "long"; +import { bindActionCreators, Store } from "redux"; +import { + DatabasesPageActions, + DatabasesPageData, + DatabasesPageDataDatabase, + DatabasesPageDataMissingTable, +} from "@cockroachlabs/cluster-ui"; + +import { AdminUIState, createAdminUIStore } from "src/redux/state"; +import * as fakeApi from "src/util/fakeApi"; +import { mapStateToProps, mapDispatchToProps } from "./redux"; + +class TestDriver { + private readonly actions: DatabasesPageActions; + private readonly properties: () => DatabasesPageData; + + constructor(store: Store) { + this.actions = bindActionCreators( + mapDispatchToProps, + store.dispatch.bind(store), + ); + this.properties = () => mapStateToProps(store.getState()); + } + + async refreshDatabases() { + return this.actions.refreshDatabases(); + } + + async refreshDatabaseDetails(database: string) { + return this.actions.refreshDatabaseDetails(database); + } + + async refreshTableStats(database: string, table: string) { + return this.actions.refreshTableStats(database, table); + } + + assertProperties(expected: DatabasesPageData) { + assert.deepEqual(this.properties(), expected); + } + + assertDatabaseProperties( + database: string, + expected: DatabasesPageDataDatabase, + ) { + assert.deepEqual(this.findDatabase(database), expected); + } + + assertMissingTableProperties( + database: string, + table: string, + expected: DatabasesPageDataMissingTable, + ) { + assert.deepEqual( + this.findMissingTable(this.findDatabase(database), table), + expected, + ); + } + + private findDatabase(name: string) { + return _.find(this.properties().databases, (row) => row.name == name); + } + + private findMissingTable(database: DatabasesPageDataDatabase, name: string) { + return _.find(database.missingTables, (table) => table.name == name); + } +} + +describe("Databases Page", function () { + let driver: TestDriver; + + beforeEach(function () { + driver = new TestDriver(createAdminUIStore(createMemoryHistory())); + }); + + afterEach(function () { + fakeApi.restore(); + }); + + it("starts in a pre-loading state", async function () { + driver.assertProperties({ + loading: false, + loaded: false, + databases: [], + }); + }); + + it("makes a row for each database", async function () { + fakeApi.stubDatabases({ + databases: ["system", "test"], + }); + + await driver.refreshDatabases(); + + driver.assertProperties({ + loading: false, + loaded: true, + databases: [ + { + loading: false, + loaded: false, + name: "system", + sizeInBytes: 0, + tableCount: 0, + rangeCount: 0, + missingTables: [], + }, + { + loading: false, + loaded: false, + name: "test", + sizeInBytes: 0, + tableCount: 0, + rangeCount: 0, + missingTables: [], + }, + ], + }); + }); + + it("fills in database details", async function () { + fakeApi.stubDatabases({ + databases: ["system", "test"], + }); + + fakeApi.stubDatabaseDetails("system", { + table_names: ["foo", "bar"], + stats: { + missing_tables: [], + range_count: new Long(3), + approximate_disk_bytes: new Long(7168), + }, + }); + + fakeApi.stubDatabaseDetails("test", { + table_names: ["widgets"], + stats: { + missing_tables: [], + range_count: new Long(42), + approximate_disk_bytes: new Long(1234), + }, + }); + + await driver.refreshDatabases(); + await driver.refreshDatabaseDetails("system"); + await driver.refreshDatabaseDetails("test"); + + driver.assertDatabaseProperties("system", { + loading: false, + loaded: true, + name: "system", + sizeInBytes: 7168, + tableCount: 2, + rangeCount: 3, + missingTables: [], + }); + + driver.assertDatabaseProperties("test", { + loading: false, + loaded: true, + name: "test", + sizeInBytes: 1234, + tableCount: 1, + rangeCount: 42, + missingTables: [], + }); + }); + + describe("fallback cases", function () { + describe("missing tables", function () { + it("exposes them so the component can refresh them", async function () { + fakeApi.stubDatabases({ + databases: ["system"], + }); + + fakeApi.stubDatabaseDetails("system", { + table_names: ["foo", "bar"], + stats: { + missing_tables: [{ name: "bar" }], + range_count: new Long(3), + approximate_disk_bytes: new Long(7168), + }, + }); + + await driver.refreshDatabases(); + await driver.refreshDatabaseDetails("system"); + + driver.assertDatabaseProperties("system", { + loading: false, + loaded: true, + name: "system", + sizeInBytes: 7168, + tableCount: 2, + rangeCount: 3, + missingTables: [{ loading: false, name: "bar" }], + }); + }); + + it("merges available individual stats into the totals", async function () { + fakeApi.stubDatabases({ + databases: ["system"], + }); + + fakeApi.stubDatabaseDetails("system", { + table_names: ["foo", "bar"], + stats: { + missing_tables: [{ name: "bar" }], + range_count: new Long(3), + approximate_disk_bytes: new Long(7168), + }, + }); + + fakeApi.stubTableStats("system", "bar", { + range_count: new Long(5), + approximate_disk_bytes: new Long(1024), + }); + + await driver.refreshDatabases(); + await driver.refreshDatabaseDetails("system"); + await driver.refreshTableStats("system", "bar"); + + driver.assertDatabaseProperties("system", { + loading: false, + loaded: true, + name: "system", + sizeInBytes: 8192, + tableCount: 2, + rangeCount: 8, + missingTables: [], + }); + }); + }); + + describe("missing stats", function () { + it("builds a list of missing tables", async function () { + fakeApi.stubDatabases({ + databases: ["system"], + }); + + fakeApi.stubDatabaseDetails("system", { + table_names: ["foo", "bar"], + }); + + await driver.refreshDatabases(); + await driver.refreshDatabaseDetails("system"); + + driver.assertDatabaseProperties("system", { + loading: false, + loaded: true, + name: "system", + sizeInBytes: 0, + tableCount: 2, + rangeCount: 0, + missingTables: [ + { loading: false, name: "foo" }, + { loading: false, name: "bar" }, + ], + }); + }); + + it("merges individual stats into the totals", async function () { + fakeApi.stubDatabases({ + databases: ["system"], + }); + + fakeApi.stubDatabaseDetails("system", { + table_names: ["foo", "bar"], + }); + + fakeApi.stubTableStats("system", "foo", { + range_count: new Long(3), + approximate_disk_bytes: new Long(7168), + }); + + fakeApi.stubTableStats("system", "bar", { + range_count: new Long(5), + approximate_disk_bytes: new Long(1024), + }); + + await driver.refreshDatabases(); + await driver.refreshDatabaseDetails("system"); + await driver.refreshTableStats("system", "foo"); + + driver.assertDatabaseProperties("system", { + loading: false, + loaded: true, + name: "system", + sizeInBytes: 7168, + tableCount: 2, + rangeCount: 3, + missingTables: [{ loading: false, name: "bar" }], + }); + + await driver.refreshTableStats("system", "bar"); + + driver.assertDatabaseProperties("system", { + loading: false, + loaded: true, + name: "system", + sizeInBytes: 8192, + tableCount: 2, + rangeCount: 8, + missingTables: [], + }); + }); + }); + }); +}); diff --git a/pkg/ui/src/views/databases/databasesPage/redux.ts b/pkg/ui/src/views/databases/databasesPage/redux.ts new file mode 100644 index 000000000000..9488d0640a19 --- /dev/null +++ b/pkg/ui/src/views/databases/databasesPage/redux.ts @@ -0,0 +1,105 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import _ from "lodash"; +import { createSelector } from "reselect"; +import { DatabasesPageData } from "@cockroachlabs/cluster-ui"; + +import { cockroach } from "src/js/protos"; +import { + generateTableID, + refreshDatabases, + refreshDatabaseDetails, + refreshTableStats, +} from "src/redux/apiReducers"; +import { AdminUIState } from "src/redux/state"; +import { FixLong } from "src/util/fixLong"; + +const { DatabaseDetailsRequest, TableStatsRequest } = cockroach.server.serverpb; + +export const mapStateToProps = createSelector( + (state: AdminUIState) => state.cachedData.databases, + (state: AdminUIState) => state.cachedData.databaseDetails, + (state: AdminUIState) => state.cachedData.tableStats, + + (databases, databaseDetails, tableStats): DatabasesPageData => { + return { + loading: databases.inFlight, + loaded: databases.valid, + databases: _.map(databases.data?.databases, (database) => { + const details = databaseDetails[database]; + + const stats = details?.data?.stats; + let sizeInBytes = FixLong( + stats?.approximate_disk_bytes || 0, + ).toNumber(); + let rangeCount = FixLong(stats?.range_count || 0).toNumber(); + + // We offer the component a chance to refresh any table-level stats we + // weren't able to gather during the initial database details call, by + // exposing a list of "missing tables." + // + // Furthermore, when the database-level stats are completely absent + // from the database details response (perhaps we're talking to an + // older backend that doesn't support them), we mark _all_ the tables + // as "missing", so that the component can trigger refresh calls for + // all of their individual stats. + + const possiblyMissingTables = stats + ? _.map(stats.missing_tables, (table) => table.name) + : details?.data?.table_names; + + const [individuallyLoadedTables, missingTables] = _.partition( + possiblyMissingTables, + (table) => { + return !!tableStats[generateTableID(database, table)]?.valid; + }, + ); + + _.each(individuallyLoadedTables, (table) => { + const stats = tableStats[generateTableID(database, table)]; + sizeInBytes += FixLong( + stats?.data?.approximate_disk_bytes || 0, + ).toNumber(); + rangeCount += FixLong(stats?.data?.range_count || 0).toNumber(); + }); + + return { + loading: !!details?.inFlight, + loaded: !!details?.valid, + name: database, + sizeInBytes: sizeInBytes, + tableCount: details?.data?.table_names?.length || 0, + rangeCount: rangeCount, + missingTables: _.map(missingTables, (table) => { + return { + loading: !!tableStats[generateTableID(database, table)]?.inFlight, + name: table, + }; + }), + }; + }), + }; + }, +); + +export const mapDispatchToProps = { + refreshDatabases, + + refreshDatabaseDetails: (database: string) => { + return refreshDatabaseDetails( + new DatabaseDetailsRequest({ database, include_stats: true }), + ); + }, + + refreshTableStats: (database: string, table: string) => { + return refreshTableStats(new TableStatsRequest({ database, table })); + }, +}; diff --git a/pkg/ui/src/views/shared/components/sql/tableInfo.tsx b/pkg/ui/src/views/shared/components/sql/tableInfo.tsx deleted file mode 100644 index 92cbf0832d8d..000000000000 --- a/pkg/ui/src/views/shared/components/sql/tableInfo.tsx +++ /dev/null @@ -1,102 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -import { Tooltip } from "antd"; -import React from "react"; -import { connect } from "react-redux"; -import { Link } from "react-router-dom"; -import classNames from "classnames/bind"; - -import * as protos from "src/js/protos"; -import { refreshTableDetails, refreshTableStats } from "src/redux/apiReducers"; -import { AdminUIState } from "src/redux/state"; -import { databaseNameAttr, tableNameAttr } from "src/util/constants"; -import { selectTableInfo } from "src/views/databases/containers/tableDetails"; -import { TableInfo } from "src/views/databases/data/tableInfo"; -import { Highlight } from "./highlight"; -import styles from "./sqlhighlight.module.styl"; - -interface TableInfoComponentProps { - title: any; - params: { - database_name: string; - table_name: string; - }; -} -interface TableInfoProps { - tableInfo: TableInfo; - refreshTableDetails: typeof refreshTableDetails; - refreshTableStats: typeof refreshTableStats; -} - -const cx = classNames.bind(styles); - -class TableInfoComponent extends React.Component< - TableInfoComponentProps & TableInfoProps -> { - componentDidMount() { - this.loadTable(); - } - - loadTable = () => { - this.props.refreshTableDetails( - new protos.cockroach.server.serverpb.TableDetailsRequest({ - database: this.props.params[databaseNameAttr], - table: this.props.params[tableNameAttr], - }), - ); - this.props.refreshTableStats( - new protos.cockroach.server.serverpb.TableStatsRequest({ - database: this.props.params[databaseNameAttr], - table: this.props.params[tableNameAttr], - }), - ); - }; - - render() { - const { title, tableInfo, params } = this.props; - return ( - - - - } - > - - {title} - - - ); - } -} - -const mapStateToProps = (state: AdminUIState, props: any) => { - return { - tableInfo: selectTableInfo(state, props), - }; -}; - -const mapDispatchToProps = { - refreshTableDetails, - refreshTableStats, -}; - -const TableInfoConnected = connect( - mapStateToProps, - mapDispatchToProps, -)(TableInfoComponent); - -export default TableInfoConnected;