From 66b955051099fa54fed3c3ca0cd8c0ac86e11163 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Tue, 24 Sep 2019 12:05:01 +0200 Subject: [PATCH] cli: fix the demo licensing code Release justification: fixes a flaky test, fixes UX of main new feature Before this patch, there were multiple problems with the code: - if the license acquisition was disabled by the env var config, the error message would not be clear. - the licensing code would deadlock silently on OSS-only builds (because the license failure channel was not written in that control branch). - the error/warning messages would be interleaved on the same line as the input line (missing newline at start of message). - the test code would fail when the license server is not available. - the set up of the example database and workload would be performed asynchronously, with unclear signalling of when the user can expect to use them interactively. After this patch: - it's possible to override the license acquisition URL with COCKROACH_DEMO_LICENSE_URL, this is used in tests. - setting up the example database, partitioning and workload is done before presenting the interactive prompt. - partitioning the example database, if requested by --geo-partitioned-replicas, waits for license acquisition to complete (license acquisition remains asynchronous otherwise). - impossible configurations are reported early(earlier). For example: - OSS-only builds: ``` kena@kenax ~/cockroach % ./cockroach demo --geo-partitioned-replicas * * ERROR: enterprise features are required for this demo, cannot run from OSS-only binary * Failed running "demo" ``` For license acquisition failures: ``` kena@kenax ~/cockroach % ./cockroach demo --geo-partitioned-replicas error while contacting licensing server: Get https://192.168.2.170/api/license?clusterid=5548b310-14b7-46de-8c92-30605bfe95c4&kind=demo&version=v19.2: net/http: request canceled while waiting for connection (Client.Timeout exceeded while awaiting headers) * * ERROR: license acquisition was unsuccessful. * Note: enterprise features are needed for --geo-partitioned-replicas. * Error: unable to acquire a license for this demo Failed running "demo" ``` Additionally, this change fixes test flakiness that arises from an unavailable license server. Release note (cli change): To enable uses of `cockroach demo` with enterprise features in firewalled network environments, it is now possible to redirect the license acquisition with the environment variable COCKROACH_DEMO_LICENSE_URL to a replacement server (for example a suitably configured HTTP proxy). --- pkg/ccl/cliccl/demo.go | 8 +- pkg/cli/context.go | 14 +- pkg/cli/demo.go | 287 ++++++++++-------- pkg/cli/interactive_tests/test_demo.tcl | 2 +- .../test_demo_partitioning.tcl | 33 +- .../interactive_tests/test_demo_telemetry.tcl | 5 + 6 files changed, 203 insertions(+), 146 deletions(-) diff --git a/pkg/ccl/cliccl/demo.go b/pkg/ccl/cliccl/demo.go index 1199f50ecbcf..f29b8e09c309 100644 --- a/pkg/ccl/cliccl/demo.go +++ b/pkg/ccl/cliccl/demo.go @@ -17,13 +17,17 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/cli" + "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" ) // This URL grants a license that is valid for 24 hours. -const licenseURL = "https://register.cockroachdb.com/api/license" +const licenseDefaultURL = "https://register.cockroachdb.com/api/license" + +// We make licenseURL configurable for use in tests. +var licenseURL = envutil.EnvOrDefaultString("COCKROACH_DEMO_LICENSE_URL", licenseDefaultURL) func getLicense(clusterID uuid.UUID) (string, error) { client := &http.Client{ @@ -60,7 +64,7 @@ func getLicense(clusterID uuid.UUID) (string, error) { func getAndApplyLicense(db *gosql.DB, clusterID uuid.UUID, org string) (bool, error) { license, err := getLicense(clusterID) if err != nil { - fmt.Fprintf(log.OrigStderr, "error when contacting licensing server: %+v\n", err) + fmt.Fprintf(log.OrigStderr, "\nerror while contacting licensing server:\n%+v\n", err) return false, nil } if _, err := db.Exec(`SET CLUSTER SETTING cluster.organization = $1`, org); err != nil { diff --git a/pkg/cli/context.go b/pkg/cli/context.go index 35eb1040a1ed..9c9d56ad0017 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -149,6 +149,8 @@ func initCLIDefaults() { demoCtx.runWorkload = false demoCtx.localities = nil demoCtx.geoPartitionedReplicas = false + demoCtx.disableTelemetry = false + demoCtx.disableLicenseAcquisition = false initPreFlagsDefaults() @@ -343,9 +345,11 @@ var sqlfmtCtx struct { // demoCtx captures the command-line parameters of the `demo` command. // Defaults set by InitCLIDefaults() above. var demoCtx struct { - nodes int - useEmptyDatabase bool - runWorkload bool - localities demoLocalityList - geoPartitionedReplicas bool + nodes int + disableTelemetry bool + disableLicenseAcquisition bool + useEmptyDatabase bool + runWorkload bool + localities demoLocalityList + geoPartitionedReplicas bool } diff --git a/pkg/cli/demo.go b/pkg/cli/demo.go index 2bd29e432b51..2aab7d81fa2d 100644 --- a/pkg/cli/demo.go +++ b/pkg/cli/demo.go @@ -32,7 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/workload" "github.com/cockroachdb/cockroach/pkg/workload/histogram" "github.com/cockroachdb/cockroach/pkg/workload/workloadsql" - "github.com/pkg/errors" + "github.com/cockroachdb/errors" "github.com/spf13/cobra" "github.com/spf13/pflag" "golang.org/x/time/rate" @@ -114,22 +114,24 @@ func init() { // The cliccl package sets this function if enterprise features are available to demo. var GetAndApplyLicense func(dbConn *gosql.DB, clusterID uuid.UUID, org string) (bool, error) -func setupTransientServers( - cmd *cobra.Command, gen workload.Generator, -) (connURL string, adminURL string, cleanup func(), err error) { - cleanup = func() {} - ctx := context.Background() +type transientCluster struct { + connURL string + stopper *stop.Stopper + s *server.TestServer + cleanup func() +} - if demoCtx.nodes <= 0 { - return "", "", cleanup, errors.Errorf("must have a positive number of nodes") - } +func setupTransientCluster( + ctx context.Context, cmd *cobra.Command, gen workload.Generator, +) (c transientCluster, err error) { + c.cleanup = func() {} // The user specified some localities for their nodes. if len(demoCtx.localities) != 0 { // Error out of localities don't line up with requested node // count before doing any sort of setup. if len(demoCtx.localities) != demoCtx.nodes { - return "", "", cleanup, errors.Errorf("number of localities specified must equal number of nodes") + return c, errors.Errorf("number of localities specified must equal number of nodes") } } else { demoCtx.localities = make([]roachpb.Locality, demoCtx.nodes) @@ -152,44 +154,43 @@ func setupTransientServers( _ = sf.Value.Set(log.Severity_NONE.String()) sf.Changed = true } - stopper, err := setupAndInitializeLoggingAndProfiling(ctx, cmd) + c.stopper, err = setupAndInitializeLoggingAndProfiling(ctx, cmd) if err != nil { - return connURL, adminURL, cleanup, err + return c, err } - cleanup = func() { stopper.Stop(ctx) } + c.cleanup = func() { c.stopper.Stop(ctx) } // Create the first transient server. The others will join this one. args := base.TestServerArgs{ PartOfCluster: true, Insecure: true, - Stopper: stopper, + Stopper: c.stopper, } serverFactory := server.TestServerFactory - var s *server.TestServer for i := 0; i < demoCtx.nodes; i++ { // All the nodes connect to the address of the first server created. - if s != nil { - args.JoinAddr = s.ServingRPCAddr() + if c.s != nil { + args.JoinAddr = c.s.ServingRPCAddr() } if demoCtx.localities != nil { args.Locality = demoCtx.localities[i] } serv := serverFactory.New(args).(*server.TestServer) if err := serv.Start(args); err != nil { - return connURL, adminURL, cleanup, err + return c, err } // Remember the first server created. if i == 0 { - s = serv + c.s = serv } } if demoCtx.nodes < 3 { // Set up the default zone configuration. We are using an in-memory store // so we really want to disable replication. - if err := cliDisableReplication(ctx, s.Server); err != nil { - return ``, ``, cleanup, err + if err := cliDisableReplication(ctx, c.s.Server); err != nil { + return c, err } } @@ -200,112 +201,107 @@ func setupTransientServers( url := url.URL{ Scheme: "postgres", User: url.User(security.RootUser), - Host: s.ServingSQLAddr(), + Host: c.s.ServingSQLAddr(), RawQuery: options.Encode(), } if gen != nil { url.Path = gen.Meta().Name } - urlStr := url.String() - - // Communicate information about license acquisition to services - // that depend on it. - licenseSuccess := make(chan bool, 1) + c.connURL = url.String() // Start up the update check loop. // We don't do this in (*server.Server).Start() because we don't want it // in tests. - if !cluster.TelemetryOptOut() { - s.PeriodicallyCheckForUpdates(ctx) + if !demoCtx.disableTelemetry { + c.s.PeriodicallyCheckForUpdates(ctx) + } + return c, nil +} + +func (c *transientCluster) setupWorkload(ctx context.Context, gen workload.Generator) error { + // Communicate information about license acquisition to services + // that depend on it. + licenseDone := make(chan struct{}) + if demoCtx.disableLicenseAcquisition { + close(licenseDone) + } else { // If we allow telemetry, then also try and get an enterprise license for the demo. // GetAndApplyLicense will be nil in the pure OSS/BSL build of cockroach. - if GetAndApplyLicense != nil { - db, err := gosql.Open("postgres", urlStr) + db, err := gosql.Open("postgres", c.connURL) + if err != nil { + return err + } + // Perform license acquisition asynchronously to avoid delay in cli startup. + go func() { + defer db.Close() + success, err := GetAndApplyLicense(db, c.s.ClusterID(), demoOrg) if err != nil { - return ``, ``, cleanup, err + exitWithError("demo", err) } - // Perform license acquisition asynchronously to avoid delay in cli startup. - go func() { - defer db.Close() - success, err := GetAndApplyLicense(db, s.ClusterID(), demoOrg) - if err != nil { - exitWithError("demo", err) - } - if !success { - const msg = "Unable to acquire demo license. Enterprise features are not enabled in this session.\n" - fmt.Fprint(stderr, msg) + if !success { + if demoCtx.geoPartitionedReplicas { + log.Shout(ctx, log.Severity_ERROR, + "license acquisition was unsuccessful.\nNote: enterprise features are needed for --geo-partitioned-replicas.") + exitWithError("demo", errors.New("unable to acquire a license for this demo")) } - licenseSuccess <- success - }() - } - } else { - // If we aren't supposed to check for a license, then automatically - // notify failure. - licenseSuccess <- false + + const msg = "\nwarning: unable to acquire demo license - enterprise features are not enabled." + fmt.Fprintln(stderr, msg) + } + close(licenseDone) + }() } // If there is a load generator, create its database and load its // fixture. if gen != nil { - db, err := gosql.Open("postgres", urlStr) + db, err := gosql.Open("postgres", c.connURL) if err != nil { - return ``, ``, cleanup, err + return err } defer db.Close() if _, err := db.Exec(`CREATE DATABASE ` + gen.Meta().Name); err != nil { - return ``, ``, cleanup, err + return err } ctx := context.TODO() var l workloadsql.InsertsDataLoader if _, err := workloadsql.Setup(ctx, db, gen, l); err != nil { - return ``, ``, cleanup, err + return err } - partitioningComplete := make(chan struct{}, 1) - // If we are requested to prepartition our data spawn a goroutine to do the partitioning. + // Perform partitioning if requested by configuration. if demoCtx.geoPartitionedReplicas { - go func() { - success := <-licenseSuccess - // Only try partitioning if license acquisition was successful. - if success { - db, err := gosql.Open("postgres", urlStr) - if err != nil { - exitWithError("demo", err) - } - defer db.Close() - // Based on validation done in setup, we know that this workload has a partitioning step. - if err := gen.(workload.Hookser).Hooks().Partition(db); err != nil { - exitWithError("demo", err) - } - partitioningComplete <- struct{}{} - } else { - const msg = "license acquisition was unsuccessful. Enterprise features are needed to partition data" - exitWithError("demo", errors.New(msg)) - } - }() + // Wait until the license has been acquired to trigger partitioning. + fmt.Println("#\n# Waiting for license acquisition to complete...") + <-licenseDone + + fmt.Println("#\n# Partitioning the demo database, please wait...") + + db, err := gosql.Open("postgres", c.connURL) + if err != nil { + return err + } + defer db.Close() + // Based on validation done in setup, we know that this workload has a partitioning step. + if err := gen.(workload.Hookser).Hooks().Partition(db); err != nil { + return errors.Wrapf(err, "partitioning the demo database") + } } + // Run the workload. This must occur after partitioning the database. if demoCtx.runWorkload { - go func() { - // If partitioning was requested, wait for that to complete before running the workload. - if demoCtx.geoPartitionedReplicas { - <-partitioningComplete - } - if err := runWorkload(ctx, gen, urlStr, stopper); err != nil { - exitWithError("demo", err) - } - }() + if err := c.runWorkload(ctx, gen); err != nil { + return errors.Wrapf(err, "starting background workload") + } } } - return urlStr, s.AdminURL(), cleanup, nil + return nil } -func runWorkload( - ctx context.Context, gen workload.Generator, dbURL string, stopper *stop.Stopper, -) error { +func (c *transientCluster) runWorkload(ctx context.Context, gen workload.Generator) error { opser, ok := gen.(workload.Opser) if !ok { return errors.Errorf("default dataset %s does not have a workload defined", gen.Meta().Name) @@ -313,7 +309,7 @@ func runWorkload( // Dummy registry to prove to the Opser. reg := histogram.NewRegistry(time.Duration(100) * time.Millisecond) - ops, err := opser.Ops([]string{dbURL}, reg) + ops, err := opser.Ops([]string{c.connURL}, reg) if err != nil { return errors.Wrap(err, "unable to create workload") } @@ -342,7 +338,7 @@ func runWorkload( } } } - stopper.RunWorker(ctx, workloadFun(workerFn)) + c.stopper.RunWorker(ctx, workloadFun(workerFn)) } return nil @@ -363,7 +359,9 @@ func incrementTelemetryCounters(cmd *cobra.Command) { } } -func runDemo(cmd *cobra.Command, gen workload.Generator) error { +func checkDemoConfiguration( + cmd *cobra.Command, gen workload.Generator, +) (workload.Generator, error) { if gen == nil && !demoCtx.useEmptyDatabase { // Use a default dataset unless prevented by --empty. gen = defaultGenerator @@ -371,57 +369,75 @@ func runDemo(cmd *cobra.Command, gen workload.Generator) error { // Make sure that the user didn't request a workload and an empty database. if demoCtx.runWorkload && demoCtx.useEmptyDatabase { - return errors.New("cannot run a workload against an empty database") + return nil, errors.New("cannot run a workload against an empty database") } - // Make sure that the user didn't request to have a topology and an empty database. - if demoCtx.geoPartitionedReplicas && demoCtx.useEmptyDatabase { - return errors.New("cannot setup geo-partitioned replicas topology on an empty database") + // Make sure the number of nodes is valid. + if demoCtx.nodes <= 0 { + return nil, errors.Newf("--nodes has invalid value (expected positive, got %d)", demoCtx.nodes) } - // Make sure that the Movr database is selected when automatically partitioning. - if demoCtx.geoPartitionedReplicas && (gen == nil || gen.Meta().Name != "movr") { - return errors.New("--geo-partitioned-replicas must be used with the Movr dataset") - } + demoCtx.disableTelemetry = cluster.TelemetryOptOut() + demoCtx.disableLicenseAcquisition = demoCtx.disableTelemetry || (GetAndApplyLicense == nil) - // If the geo-partitioned replicas flag was given and the demo localities have changed, throw an error. - if demoCtx.geoPartitionedReplicas && demoCtx.localities != nil { - return errors.New("--demo-locality cannot be used with --geo-partitioned-replicas") - } + if demoCtx.geoPartitionedReplicas { + if demoCtx.disableLicenseAcquisition { + return nil, errors.New("enterprise features are needed for this demo (--geo-partitioning-replicas)") + } - // If the geo-partitioned replicas flag was given and the nodes have changed, throw an error. - if demoCtx.geoPartitionedReplicas && flagSetForCmd(cmd).Lookup(cliflags.DemoNodes.Name).Changed { - return errors.New("--nodes cannot be used with --geo-partitioned-replicas") - } + // Make sure that the user didn't request to have a topology and an empty database. + if demoCtx.useEmptyDatabase { + return nil, errors.New("cannot setup geo-partitioned replicas topology on an empty database") + } - // If geo-partition-replicas is requested, make sure the workload has a Partitioning step. - if demoCtx.geoPartitionedReplicas { + // Make sure that the Movr database is selected when automatically partitioning. + if gen == nil || gen.Meta().Name != "movr" { + return nil, errors.New("--geo-partitioned-replicas must be used with the Movr dataset") + } + + // If the geo-partitioned replicas flag was given and the demo localities have changed, throw an error. + if demoCtx.localities != nil { + return nil, errors.New("--demo-locality cannot be used with --geo-partitioned-replicas") + } + + // If the geo-partitioned replicas flag was given and the nodes have changed, throw an error. + if flagSetForCmd(cmd).Lookup(cliflags.DemoNodes.Name).Changed { + if demoCtx.nodes != 9 { + return nil, errors.New("--nodes with a value different from 9 cannot be used with --geo-partitioned-replicas") + } + } else { + const msg = `# +# --geo-partitioned replicas operates on a 9 node cluster. +# The cluster size has been changed from the default to 9 nodes.` + fmt.Println(msg) + demoCtx.nodes = 9 + } + + // If geo-partition-replicas is requested, make sure the workload has a Partitioning step. configErr := errors.New(fmt.Sprintf("workload %s is not configured to have a partitioning step", gen.Meta().Name)) hookser, ok := gen.(workload.Hookser) if !ok { - return configErr + return nil, configErr } if hookser.Hooks().Partition == nil { - return configErr + return nil, configErr } } + return gen, nil +} + +func runDemo(cmd *cobra.Command, gen workload.Generator) (err error) { + if gen, err = checkDemoConfiguration(cmd, gen); err != nil { + return err + } // Record some telemetry about what flags are being used. incrementTelemetryCounters(cmd) - // Th geo-partitioned replicas demo only works on a 9 node cluster, so set the node count as such. - // Ignore input user localities so that the nodes have the same attributes/localities as expected. - if demoCtx.geoPartitionedReplicas { - const msg = `# -# --geo-partitioned replicas operates on a 9 node cluster. -# The cluster size has been changed from the default to 9 nodes.` - fmt.Println(msg) - demoCtx.nodes = 9 - demoCtx.localities = nil - } + ctx := context.Background() - connURL, adminURL, cleanup, err := setupTransientServers(cmd, gen) - defer cleanup() + c, err := setupTransientCluster(ctx, cmd, gen) + defer c.cleanup() if err != nil { return checkAndMaybeShout(err) } @@ -435,22 +451,39 @@ func runDemo(cmd *cobra.Command, gen workload.Generator) error { # You are connected to a temporary, in-memory CockroachDB cluster of %d node%s. `, demoCtx.nodes, util.Pluralize(int64(demoCtx.nodes))) + if demoCtx.disableTelemetry { + fmt.Println("#\n# Telemetry and automatic license acquisition disabled by configuration.") + } else if demoCtx.disableLicenseAcquisition { + fmt.Println("#\n# Enterprise features disabled by OSS-only build.") + } else { + fmt.Println("#\n# This demo session will attempt to enable enterprise features\n" + + "# by acquiring a temporary license from Cockroach Labs in the background.\n" + + "# To disable this behavior, set the environment variable\n" + + "# COCKROACH_SKIP_ENABLING_DIAGNOSTIC_REPORTING=true.") + } + } + + if err := c.setupWorkload(ctx, gen); err != nil { + return err + } + + if cliCtx.isInteractive { if gen != nil { - fmt.Printf("# The cluster has been preloaded with the %q dataset\n# (%s).\n", + fmt.Printf("#\n# The cluster has been preloaded with the %q dataset\n# (%s).\n", gen.Meta().Name, gen.Meta().Description) } fmt.Printf(`# -# Your changes will not be saved! +# Reminder: your changes to data stored in the demo session will not be saved! # # Web UI: %s # -`, adminURL) +`, c.s.AdminURL()) } - checkTzDatabaseAvailability(context.Background()) + checkTzDatabaseAvailability(ctx) - conn := makeSQLConn(connURL) + conn := makeSQLConn(c.connURL) defer conn.Close() return runClient(cmd, conn) diff --git a/pkg/cli/interactive_tests/test_demo.tcl b/pkg/cli/interactive_tests/test_demo.tcl index ab753eb78893..333fca5e06b0 100644 --- a/pkg/cli/interactive_tests/test_demo.tcl +++ b/pkg/cli/interactive_tests/test_demo.tcl @@ -7,7 +7,7 @@ spawn $argv demo # Be polite. eexpect "Welcome" # Warn the user that they won't get persistence. -eexpect "Your changes will not be saved!" +eexpect "your changes to data stored in the demo session will not be saved!" # Inform the necessary URL. eexpect "Web UI: http:" # Ensure same messages as cockroach sql. diff --git a/pkg/cli/interactive_tests/test_demo_partitioning.tcl b/pkg/cli/interactive_tests/test_demo_partitioning.tcl index bb7bf140d416..28527a871deb 100644 --- a/pkg/cli/interactive_tests/test_demo_partitioning.tcl +++ b/pkg/cli/interactive_tests/test_demo_partitioning.tcl @@ -7,18 +7,20 @@ start_test "Expect partitioning succeeds" spawn $argv demo --geo-partitioned-replicas # wait for the shell to start up -eexpect "movr>" - -# send multiple "SHOW PARTITIONS" requests to the DB as partitioning is happen asynchronously. -for {set i 0} {$i < 10} {incr i} { - send "SELECT count(*) FROM \[SHOW PARTITIONS FROM DATABASE movr\];\r" - sleep 1 +expect { + "Timeout exceeded" { + # The license server is unreachable. There's not much we can test here. + # Simply ignore the test. + report "License server could not be reached - skipping with no error" + exit 0 + } + "movr>" {} } -# The number of partitions across the MovR database we expect is 24. +send "SELECT count(*) AS NRPARTS FROM \[SHOW PARTITIONS FROM DATABASE movr\];\r" +eexpect "nrparts" eexpect "24" eexpect "(1 row)" -eexpect "movr>" send "SHOW PARTITIONS FROM TABLE vehicles;\r" @@ -74,14 +76,23 @@ end_test start_test "Expect an error if geo-partitioning is requested and a license cannot be acquired" +set env(COCKROACH_DEMO_LICENSE_URL) "https://127.0.0.1:9999/" +spawn $argv demo --geo-partitioned-replicas +eexpect "error while contacting licensing server" +eexpect "dial tcp" +eexpect "ERROR: license acquisition was unsuccessful" +eexpect eof +end_test + +start_test "Expect an error if geo-partitioning is requested and license acquisition is disabled" # set the proper environment variable set env(COCKROACH_SKIP_ENABLING_DIAGNOSTIC_REPORTING) "true" spawn $argv demo --geo-partitioned-replicas # expect a failure -eexpect "Error: license acquisition was unsuccessful. Enterprise features are needed to partition data" +eexpect Error +eexpect "enterprise features are needed for this demo" # clean up after the test -interrupt eexpect eof -end_test +end_test diff --git a/pkg/cli/interactive_tests/test_demo_telemetry.tcl b/pkg/cli/interactive_tests/test_demo_telemetry.tcl index c64be5515ab1..43ad6a3a8689 100644 --- a/pkg/cli/interactive_tests/test_demo_telemetry.tcl +++ b/pkg/cli/interactive_tests/test_demo_telemetry.tcl @@ -7,6 +7,10 @@ start_test "Check cockroach demo telemetry and license check can be disabled" # set the proper environment variable set env(COCKROACH_SKIP_ENABLING_DIAGNOSTIC_REPORTING) "true" spawn $argv demo + +# Expect an informational message. +eexpect "Telemetry and automatic license acquisition disabled by configuration." + # wait for the CLI to start up eexpect "movr>" # send a request for an enterprise feature @@ -16,4 +20,5 @@ eexpect "use of partitions requires an enterprise license" # clean up after the test interrupt eexpect eof + end_test