github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/workload/tpch/tpch.go (about) 1 // Copyright 2018 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 package tpch 12 13 import ( 14 "context" 15 gosql "database/sql" 16 "fmt" 17 "math" 18 "strconv" 19 "strings" 20 "sync" 21 22 "github.com/cockroachdb/cockroach/pkg/util/log" 23 "github.com/cockroachdb/cockroach/pkg/util/timeutil" 24 "github.com/cockroachdb/cockroach/pkg/workload" 25 "github.com/cockroachdb/cockroach/pkg/workload/histogram" 26 "github.com/cockroachdb/errors" 27 "github.com/spf13/pflag" 28 "golang.org/x/exp/rand" 29 ) 30 31 const ( 32 numNation = 25 33 numRegion = 5 34 numPartPerSF = 200000 35 numPartSuppPerPart = 4 36 numSupplierPerSF = 10000 37 numCustomerPerSF = 150000 38 numOrderPerCustomer = 10 39 numLineItemPerSF = 6001215 40 ) 41 42 // wrongOutputError indicates that incorrect results were returned for one of 43 // the TPCH queries. 44 type wrongOutputError struct { 45 error 46 } 47 48 // TPCHWrongOutputErrorPrefix is the string that all errors about the wrong 49 // output will be prefixed with. 50 const TPCHWrongOutputErrorPrefix = "TPCH wrong output " 51 52 func (e wrongOutputError) Error() string { 53 return TPCHWrongOutputErrorPrefix + e.error.Error() 54 } 55 56 type tpch struct { 57 flags workload.Flags 58 connFlags *workload.ConnFlags 59 60 seed uint64 61 scaleFactor int 62 fks bool 63 64 disableChecks bool 65 vectorize string 66 verbose bool 67 68 queriesRaw string 69 selectedQueries []int 70 71 textPool textPool 72 localsPool *sync.Pool 73 } 74 75 func init() { 76 workload.Register(tpchMeta) 77 } 78 79 // FromScaleFactor returns a tpch generator pre-configured with the specified 80 // scale factor. 81 func FromScaleFactor(scaleFactor int) workload.Generator { 82 return workload.FromFlags(tpchMeta, fmt.Sprintf(`--scale-factor=%d`, scaleFactor)) 83 } 84 85 var tpchMeta = workload.Meta{ 86 Name: `tpch`, 87 Description: `TPC-H is a read-only workload of "analytics" queries on large datasets.`, 88 Version: `1.0.0`, 89 New: func() workload.Generator { 90 g := &tpch{} 91 g.flags.FlagSet = pflag.NewFlagSet(`tpch`, pflag.ContinueOnError) 92 g.flags.Meta = map[string]workload.FlagMeta{ 93 `queries`: {RuntimeOnly: true}, 94 `dist-sql`: {RuntimeOnly: true}, 95 `disable-checks`: {RuntimeOnly: true}, 96 `vectorize`: {RuntimeOnly: true}, 97 } 98 g.flags.Uint64Var(&g.seed, `seed`, 1, `Random number generator seed`) 99 g.flags.IntVar(&g.scaleFactor, `scale-factor`, 1, 100 `Linear scale of how much data to use (each SF is ~1GB)`) 101 g.flags.BoolVar(&g.fks, `fks`, true, `Add the foreign keys`) 102 g.flags.StringVar(&g.queriesRaw, `queries`, 103 `1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22`, 104 `Queries to run. Use a comma separated list of query numbers`) 105 g.flags.BoolVar(&g.disableChecks, `disable-checks`, false, 106 "Disable checking the output against the expected rows (default false). "+ 107 "Note that the checks are only supported for scale factor 1") 108 g.flags.StringVar(&g.vectorize, `vectorize`, `on`, 109 `Set vectorize session variable`) 110 g.flags.BoolVar(&g.verbose, `verbose`, false, 111 `Prints out the queries being run as well as histograms`) 112 g.connFlags = workload.NewConnFlags(&g.flags) 113 return g 114 }, 115 } 116 117 // Meta implements the Generator interface. 118 func (*tpch) Meta() workload.Meta { return tpchMeta } 119 120 // Flags implements the Flagser interface. 121 func (w *tpch) Flags() workload.Flags { return w.flags } 122 123 // Hooks implements the Hookser interface. 124 func (w *tpch) Hooks() workload.Hooks { 125 return workload.Hooks{ 126 Validate: func() error { 127 if w.scaleFactor != 1 { 128 fmt.Printf("check for expected rows is only supported with " + 129 "scale factor 1, so it was disabled\n") 130 w.disableChecks = true 131 } 132 for _, queryName := range strings.Split(w.queriesRaw, `,`) { 133 queryNum, err := strconv.Atoi(queryName) 134 if err != nil { 135 return err 136 } 137 if _, ok := QueriesByNumber[queryNum]; !ok { 138 return errors.Errorf(`unknown query: %s`, queryName) 139 } 140 w.selectedQueries = append(w.selectedQueries, queryNum) 141 } 142 return nil 143 }, 144 PostLoad: func(db *gosql.DB) error { 145 if w.fks { 146 // We avoid validating foreign keys because we just generated the data 147 // set and don't want to scan over the entire thing again. 148 // Unfortunately, this means that we leave the foreign keys unvalidated 149 // for the duration of the test, so the SQL optimizer can't use them. 150 // 151 // TODO(lucy-zhang): expose an internal knob to validate fk relations 152 // without performing full validation. See #38833. 153 fkStmts := []string{ 154 `ALTER TABLE nation ADD CONSTRAINT nation_fkey_region FOREIGN KEY (n_regionkey) REFERENCES region (r_regionkey) NOT VALID`, 155 `ALTER TABLE supplier ADD CONSTRAINT supplier_fkey_nation FOREIGN KEY (s_nationkey) REFERENCES nation (n_nationkey) NOT VALID`, 156 `ALTER TABLE partsupp ADD CONSTRAINT partsupp_fkey_part FOREIGN KEY (ps_partkey) REFERENCES part (p_partkey) NOT VALID`, 157 `ALTER TABLE partsupp ADD CONSTRAINT partsupp_fkey_supplier FOREIGN KEY (ps_suppkey) REFERENCES supplier (s_suppkey) NOT VALID`, 158 `ALTER TABLE customer ADD CONSTRAINT customer_fkey_nation FOREIGN KEY (c_nationkey) REFERENCES nation (n_nationkey) NOT VALID`, 159 `ALTER TABLE orders ADD CONSTRAINT orders_fkey_customer FOREIGN KEY (o_custkey) REFERENCES customer (c_custkey) NOT VALID`, 160 `ALTER TABLE lineitem ADD CONSTRAINT lineitem_fkey_orders FOREIGN KEY (l_orderkey) REFERENCES orders (o_orderkey) NOT VALID`, 161 `ALTER TABLE lineitem ADD CONSTRAINT lineitem_fkey_part FOREIGN KEY (l_partkey) REFERENCES part (p_partkey) NOT VALID`, 162 `ALTER TABLE lineitem ADD CONSTRAINT lineitem_fkey_supplier FOREIGN KEY (l_suppkey) REFERENCES supplier (s_suppkey) NOT VALID`, 163 // TODO(andyk): This fails with `pq: column "l_partkey" cannot be used 164 // by multiple foreign key constraints`. This limitation would appear 165 // to violate TPCH rules, as all foreign keys must be defined, or none 166 // at all. 167 // `ALTER TABLE lineitem ADD CONSTRAINT lineitem_fkey_partsupp FOREIGN KEY (l_partkey, l_suppkey) REFERENCES partsupp (ps_partkey, ps_suppkey) NOT VALID`, 168 } 169 170 for _, fkStmt := range fkStmts { 171 if _, err := db.Exec(fkStmt); err != nil { 172 // If the statement failed because the fk already exists, ignore it. 173 // Return the error for any other reason. 174 const duplFKErr = "columns cannot be used by multiple foreign key constraints" 175 if !strings.Contains(err.Error(), duplFKErr) { 176 return errors.Wrapf(err, "while executing %s", fkStmt) 177 } 178 } 179 } 180 } 181 return nil 182 }, 183 } 184 } 185 186 type generateLocals struct { 187 rng *rand.Rand 188 189 // namePerm is a slice of ordinals into randPartNames. 190 namePerm []int 191 192 orderData *orderSharedRandomData 193 } 194 195 // Tables implements the Generator interface. 196 func (w *tpch) Tables() []workload.Table { 197 if w.localsPool == nil { 198 w.localsPool = &sync.Pool{ 199 New: func() interface{} { 200 namePerm := make([]int, len(randPartNames)) 201 for i := range namePerm { 202 namePerm[i] = i 203 } 204 return &generateLocals{ 205 rng: rand.New(rand.NewSource(uint64(timeutil.Now().UnixNano()))), 206 namePerm: namePerm, 207 orderData: &orderSharedRandomData{ 208 partKeys: make([]int, 0, 7), 209 shipDates: make([]int64, 0, 7), 210 quantities: make([]float32, 0, 7), 211 discount: make([]float32, 0, 7), 212 tax: make([]float32, 0, 7), 213 }, 214 } 215 }, 216 } 217 } 218 219 // TODO(dan): Make this a flag that points at an official pool.txt? 220 w.textPool = &fakeTextPool{seed: w.seed} 221 222 nation := workload.Table{ 223 Name: `nation`, 224 Schema: tpchNationSchema, 225 InitialRows: workload.BatchedTuples{ 226 NumBatches: numNation, 227 FillBatch: w.tpchNationInitialRowBatch, 228 }, 229 } 230 region := workload.Table{ 231 Name: `region`, 232 Schema: tpchRegionSchema, 233 InitialRows: workload.BatchedTuples{ 234 NumBatches: numRegion, 235 FillBatch: w.tpchRegionInitialRowBatch, 236 }, 237 } 238 numSupplier := numSupplierPerSF * w.scaleFactor 239 supplier := workload.Table{ 240 Name: `supplier`, 241 Schema: tpchSupplierSchema, 242 InitialRows: workload.BatchedTuples{ 243 NumBatches: numSupplier, 244 FillBatch: w.tpchSupplierInitialRowBatch, 245 }, 246 } 247 numPart := numPartPerSF * w.scaleFactor 248 part := workload.Table{ 249 Name: `part`, 250 Schema: tpchPartSchema, 251 InitialRows: workload.BatchedTuples{ 252 NumBatches: numPart, 253 FillBatch: w.tpchPartInitialRowBatch, 254 }, 255 } 256 partsupp := workload.Table{ 257 Name: `partsupp`, 258 Schema: tpchPartSuppSchema, 259 InitialRows: workload.BatchedTuples{ 260 // 1 batch per part, hence numPartPerSF and not numPartSuppPerSF. 261 NumBatches: numPart, 262 FillBatch: w.tpchPartSuppInitialRowBatch, 263 }, 264 } 265 numCustomer := numCustomerPerSF * w.scaleFactor 266 customer := workload.Table{ 267 Name: `customer`, 268 Schema: tpchCustomerSchema, 269 InitialRows: workload.BatchedTuples{ 270 NumBatches: numCustomer, 271 FillBatch: w.tpchCustomerInitialRowBatch, 272 }, 273 } 274 orders := workload.Table{ 275 Name: `orders`, 276 Schema: tpchOrdersSchema, 277 InitialRows: workload.BatchedTuples{ 278 // 1 batch per customer. 279 NumBatches: numCustomer, 280 FillBatch: w.tpchOrdersInitialRowBatch, 281 }, 282 } 283 lineitem := workload.Table{ 284 Name: `lineitem`, 285 Schema: tpchLineItemSchema, 286 InitialRows: workload.BatchedTuples{ 287 // 1 batch per customer. 288 NumBatches: numCustomer, 289 FillBatch: w.tpchLineItemInitialRowBatch, 290 }, 291 } 292 293 return []workload.Table{ 294 nation, region, part, supplier, partsupp, customer, orders, lineitem, 295 } 296 } 297 298 // Ops implements the Opser interface. 299 func (w *tpch) Ops(urls []string, reg *histogram.Registry) (workload.QueryLoad, error) { 300 sqlDatabase, err := workload.SanitizeUrls(w, w.connFlags.DBOverride, urls) 301 if err != nil { 302 return workload.QueryLoad{}, err 303 } 304 db, err := gosql.Open(`cockroach`, strings.Join(urls, ` `)) 305 if err != nil { 306 return workload.QueryLoad{}, err 307 } 308 // Allow a maximum of concurrency+1 connections to the database. 309 db.SetMaxOpenConns(w.connFlags.Concurrency + 1) 310 db.SetMaxIdleConns(w.connFlags.Concurrency + 1) 311 312 ql := workload.QueryLoad{SQLDatabase: sqlDatabase} 313 for i := 0; i < w.connFlags.Concurrency; i++ { 314 worker := &worker{ 315 config: w, 316 hists: reg.GetHandle(), 317 db: db, 318 } 319 ql.WorkerFns = append(ql.WorkerFns, worker.run) 320 } 321 return ql, nil 322 } 323 324 type worker struct { 325 config *tpch 326 hists *histogram.Histograms 327 db *gosql.DB 328 ops int 329 } 330 331 func (w *worker) run(ctx context.Context) error { 332 queryNum := w.config.selectedQueries[w.ops%len(w.config.selectedQueries)] 333 w.ops++ 334 335 query := fmt.Sprintf("SET vectorize = '%s'; %s", w.config.vectorize, QueriesByNumber[queryNum]) 336 337 vals := make([]interface{}, maxCols) 338 for i := range vals { 339 vals[i] = new(interface{}) 340 } 341 342 start := timeutil.Now() 343 rows, err := w.db.Query(query) 344 if rows != nil { 345 defer rows.Close() 346 } 347 if err != nil { 348 return errors.Errorf("[q%d]: %s", queryNum, err) 349 } 350 var numRows int 351 // NOTE: we should *NOT* return an error from this function right away 352 // because we might get another, more meaningful error from rows.Err() which 353 // can only be accessed after we fully consumed the rows. 354 checkExpectedOutput := func() error { 355 for rows.Next() { 356 if !w.config.disableChecks { 357 if _, checkOnlyRowCount := numExpectedRowsByQueryNumber[queryNum]; !checkOnlyRowCount { 358 if err = rows.Scan(vals[:numColsByQueryNumber[queryNum]]...); err != nil { 359 return errors.Errorf("[q%d]: %s", queryNum, err) 360 } 361 362 expectedRow := expectedRowsByQueryNumber[queryNum][numRows] 363 for i, expectedValue := range expectedRow { 364 if val := *vals[i].(*interface{}); val != nil { 365 var actualValue string 366 // Currently, lib/pq for query 12 in the second and third columns 367 // (which are decimals) returns []byte. In order to compare it 368 // against our expected string value, we have this special case. 369 if byteArray, ok := val.([]byte); ok { 370 actualValue = string(byteArray) 371 } else { 372 actualValue = fmt.Sprint(val) 373 } 374 if strings.Compare(expectedValue, actualValue) != 0 { 375 var expectedFloat, actualFloat float64 376 var expectedFloatRounded, actualFloatRounded float64 377 expectedFloat, err = strconv.ParseFloat(expectedValue, 64) 378 if err != nil { 379 return errors.Errorf("[q%d] failed parsing expected value as float64 with %s\n"+ 380 "wrong result in row %d in column %d: got %q, expected %q", 381 queryNum, err, numRows, i, actualValue, expectedValue) 382 } 383 actualFloat, err = strconv.ParseFloat(actualValue, 64) 384 if err != nil { 385 return errors.Errorf("[q%d] failed parsing actual value as float64 with %s\n"+ 386 "wrong result in row %d in column %d: got %q, expected %q", 387 queryNum, err, numRows, i, actualValue, expectedValue) 388 } 389 // TPC-H spec requires 0.01 precision for DECIMALs, so we will 390 // first round the values to use in the comparison. Note that we 391 // round to a thousandth so that values like 0.601 and 0.609 were 392 // always considered to differ by less than 0.01 (due to the 393 // nature of representation of floats, it is possible that those 394 // two values when rounded to a hundredth would be represented as 395 // something like 0.59999 and 0.610001 which differ by more than 396 // 0.01). 397 expectedFloatRounded, err = strconv.ParseFloat(fmt.Sprintf("%.3f", expectedFloat), 64) 398 if err != nil { 399 return errors.Errorf("[q%d] failed parsing rounded expected value as float64 with %s\n"+ 400 "wrong result in row %d in column %d: got %q, expected %q", 401 queryNum, err, numRows, i, actualValue, expectedValue) 402 } 403 actualFloatRounded, err = strconv.ParseFloat(fmt.Sprintf("%.3f", actualFloat), 64) 404 if err != nil { 405 return errors.Errorf("[q%d] failed parsing rounded actual value as float64 with %s\n"+ 406 "wrong result in row %d in column %d: got %q, expected %q", 407 queryNum, err, numRows, i, actualValue, expectedValue) 408 } 409 if math.Abs(expectedFloatRounded-actualFloatRounded) > 0.02 { 410 // We only fail the check if the difference is more than 0.02 411 // although TPC-H spec requires 0.01 precision for DECIMALs. We 412 // are using the expected value that might not be "precisely 413 // correct." It is possible for the following situation to 414 // occur: 415 // expected < "ideal" < actual 416 // "ideal" - expected < 0.01 && actual - "ideal" < 0.01 417 // so in the worst case, actual and expected might differ by 418 // 0.02 and still be considered correct. 419 return errors.Errorf("[q%d] %f and %f differ by more than 0.02\n"+ 420 "wrong result in row %d in column %d: got %q, expected %q", 421 queryNum, actualFloatRounded, expectedFloatRounded, 422 numRows, i, actualValue, expectedValue) 423 } 424 } 425 } 426 } 427 } 428 } 429 numRows++ 430 } 431 return nil 432 } 433 434 expectedOutputError := checkExpectedOutput() 435 436 // In order to definitely get the error below, we need to fully consume the 437 // result set. 438 for rows.Next() { 439 } 440 441 // We first check whether there is any error that came from the server (for 442 // example, an out of memory error). If there is, we return it. 443 if err := rows.Err(); err != nil { 444 return errors.Errorf("[q%d]: %s", queryNum, err) 445 } 446 // Now we check whether there was an error while consuming the rows. 447 if expectedOutputError != nil { 448 return wrongOutputError{error: expectedOutputError} 449 } 450 if !w.config.disableChecks { 451 numRowsExpected, checkOnlyRowCount := numExpectedRowsByQueryNumber[queryNum] 452 if checkOnlyRowCount && numRows != numRowsExpected { 453 return wrongOutputError{ 454 error: errors.Errorf( 455 "[q%d] returned wrong number of rows: got %d, expected %d", 456 queryNum, numRows, numRowsExpected, 457 )} 458 } 459 } 460 elapsed := timeutil.Since(start) 461 if w.config.verbose { 462 w.hists.Get(fmt.Sprintf("%d", queryNum)).Record(elapsed) 463 // Note: if you are changing the output format here, please change the 464 // regex in roachtest/tpchvec.go accordingly. 465 log.Infof(ctx, "[q%d] returned %d rows after %4.2f seconds:\n%s", 466 queryNum, numRows, elapsed.Seconds(), query) 467 } else { 468 // Note: if you are changing the output format here, please change the 469 // regex in roachtest/tpchvec.go accordingly. 470 log.Infof(ctx, "[q%d] returned %d rows after %4.2f seconds", 471 queryNum, numRows, elapsed.Seconds()) 472 } 473 return nil 474 } 475 476 const ( 477 tpchNationSchema = `( 478 n_nationkey INTEGER NOT NULL PRIMARY KEY, 479 n_name CHAR(25) NOT NULL, 480 n_regionkey INTEGER NOT NULL, 481 n_comment VARCHAR(152), 482 INDEX n_rk (n_regionkey ASC) 483 )` 484 tpchRegionSchema = `( 485 r_regionkey INTEGER NOT NULL PRIMARY KEY, 486 r_name CHAR(25) NOT NULL, 487 r_comment VARCHAR(152) 488 )` 489 tpchPartSchema = `( 490 p_partkey INTEGER NOT NULL PRIMARY KEY, 491 p_name VARCHAR(55) NOT NULL, 492 p_mfgr CHAR(25) NOT NULL, 493 p_brand CHAR(10) NOT NULL, 494 p_type VARCHAR(25) NOT NULL, 495 p_size INTEGER NOT NULL, 496 p_container CHAR(10) NOT NULL, 497 p_retailprice FLOAT NOT NULL, 498 p_comment VARCHAR(23) NOT NULL 499 )` 500 tpchSupplierSchema = `( 501 s_suppkey INTEGER NOT NULL PRIMARY KEY, 502 s_name CHAR(25) NOT NULL, 503 s_address VARCHAR(40) NOT NULL, 504 s_nationkey INTEGER NOT NULL, 505 s_phone CHAR(15) NOT NULL, 506 s_acctbal FLOAT NOT NULL, 507 s_comment VARCHAR(101) NOT NULL, 508 INDEX s_nk (s_nationkey ASC) 509 )` 510 tpchPartSuppSchema = `( 511 ps_partkey INTEGER NOT NULL, 512 ps_suppkey INTEGER NOT NULL, 513 ps_availqty INTEGER NOT NULL, 514 ps_supplycost FLOAT NOT NULL, 515 ps_comment VARCHAR(199) NOT NULL, 516 PRIMARY KEY (ps_partkey ASC, ps_suppkey ASC), 517 INDEX ps_sk (ps_suppkey ASC) 518 )` 519 tpchCustomerSchema = `( 520 c_custkey INTEGER NOT NULL PRIMARY KEY, 521 c_name VARCHAR(25) NOT NULL, 522 c_address VARCHAR(40) NOT NULL, 523 c_nationkey INTEGER NOT NULL, 524 c_phone CHAR(15) NOT NULL, 525 c_acctbal FLOAT NOT NULL, 526 c_mktsegment CHAR(10) NOT NULL, 527 c_comment VARCHAR(117) NOT NULL, 528 INDEX c_nk (c_nationkey ASC) 529 )` 530 tpchOrdersSchema = `( 531 o_orderkey INTEGER NOT NULL PRIMARY KEY, 532 o_custkey INTEGER NOT NULL, 533 o_orderstatus CHAR(1) NOT NULL, 534 o_totalprice FLOAT NOT NULL, 535 o_orderdate DATE NOT NULL, 536 o_orderpriority CHAR(15) NOT NULL, 537 o_clerk CHAR(15) NOT NULL, 538 o_shippriority INTEGER NOT NULL, 539 o_comment VARCHAR(79) NOT NULL, 540 INDEX o_ck (o_custkey ASC), 541 INDEX o_od (o_orderdate ASC) 542 )` 543 tpchLineItemSchema = `( 544 l_orderkey INTEGER NOT NULL, 545 l_partkey INTEGER NOT NULL, 546 l_suppkey INTEGER NOT NULL, 547 l_linenumber INTEGER NOT NULL, 548 l_quantity FLOAT NOT NULL, 549 l_extendedprice FLOAT NOT NULL, 550 l_discount FLOAT NOT NULL, 551 l_tax FLOAT NOT NULL, 552 l_returnflag CHAR(1) NOT NULL, 553 l_linestatus CHAR(1) NOT NULL, 554 l_shipdate DATE NOT NULL, 555 l_commitdate DATE NOT NULL, 556 l_receiptdate DATE NOT NULL, 557 l_shipinstruct CHAR(25) NOT NULL, 558 l_shipmode CHAR(10) NOT NULL, 559 l_comment VARCHAR(44) NOT NULL, 560 PRIMARY KEY (l_orderkey, l_linenumber), 561 INDEX l_ok (l_orderkey ASC), 562 INDEX l_pk (l_partkey ASC), 563 INDEX l_sk (l_suppkey ASC), 564 INDEX l_sd (l_shipdate ASC), 565 INDEX l_cd (l_commitdate ASC), 566 INDEX l_rd (l_receiptdate ASC), 567 INDEX l_pk_sk (l_partkey ASC, l_suppkey ASC), 568 INDEX l_sk_pk (l_suppkey ASC, l_partkey ASC) 569 )` 570 )