diff --git a/pkg/cli/cliflags/flags.go b/pkg/cli/cliflags/flags.go index cbe67745bcc7..c35d6b1c8d07 100644 --- a/pkg/cli/cliflags/flags.go +++ b/pkg/cli/cliflags/flags.go @@ -147,6 +147,21 @@ percentage of physical memory (e.g. .25). If left unspecified, defaults to 25% o physical memory.`, } + TSDBMem = FlagInfo{ + Name: "max-tsdb-memory", + Description: ` +Maximum memory capacity available to store temporary data for use by the +time-series database to display metrics in the DB Console. Accepts numbers +interpreted as bytes, size suffixes (e.g. 1GB and 1GiB) or a +percentage of physical memory (e.g. 0.01). If left unspecified, defaults to +1% of physical memory or 64MiB whichever is greater. It maybe necessary to +manually increase this value on a cluster with hundreds of nodes where +individual nodes have very limited memory available. This can constrain +the ability of the DB Console to process time-series queries used to render +metrics for the entire cluster. This capacity constraint does not affect +SQL query execution.`, + } + SQLTempStorage = FlagInfo{ Name: "max-disk-temp-storage", Description: ` diff --git a/pkg/cli/context.go b/pkg/cli/context.go index a227a2586af8..25c3da5d0621 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/ts" "github.com/cockroachdb/cockroach/pkg/util/log/logconfig" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -128,6 +129,16 @@ func setServerContextDefaults() { if bytes, _ := memoryPercentResolver(25); bytes != 0 { serverCfg.SQLConfig.MemoryPoolSize = bytes } + + // Attempt to set serverCfg.TimeSeriesServerConfig.QueryMemoryMax to + // the default (64MiB) or 1% of system memory, whichever is greater. + if bytes, _ := memoryPercentResolver(1); bytes != 0 { + if bytes > ts.DefaultQueryMemoryMax { + serverCfg.TimeSeriesServerConfig.QueryMemoryMax = bytes + } else { + serverCfg.TimeSeriesServerConfig.QueryMemoryMax = ts.DefaultQueryMemoryMax + } + } } // baseCfg points to the base.Config inside serverCfg. diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index dac02eaa2614..8a5e64b562ba 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -526,6 +526,7 @@ func init() { // Engine flags. varFlag(f, cacheSizeValue, cliflags.Cache) varFlag(f, sqlSizeValue, cliflags.SQLMem) + varFlag(f, tsdbSizeValue, cliflags.TSDBMem) // N.B. diskTempStorageSizeValue.ResolvePercentage() will be called after // the stores flag has been parsed and the storage device that a percentage // refers to becomes known. @@ -986,6 +987,7 @@ func init() { // Engine flags. varFlag(f, sqlSizeValue, cliflags.SQLMem) + varFlag(f, tsdbSizeValue, cliflags.TSDBMem) // N.B. diskTempStorageSizeValue.ResolvePercentage() will be called after // the stores flag has been parsed and the storage device that a percentage // refers to becomes known. diff --git a/pkg/cli/flags_test.go b/pkg/cli/flags_test.go index 74aef5eba115..993b7fc29955 100644 --- a/pkg/cli/flags_test.go +++ b/pkg/cli/flags_test.go @@ -140,51 +140,61 @@ func TestClusterNameFlag(t *testing.T) { } } -func TestSQLMemoryPoolFlagValue(t *testing.T) { +func TestMemoryPoolFlagValues(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - // Avoid leaking configuration changes after the test ends. - defer initCLIDefaults() - - f := startCmd.Flags() - - // Check absolute values. - testCases := []struct { - value string - expected int64 + for _, tc := range []struct { + flag string + config *int64 }{ - {"100MB", 100 * 1000 * 1000}, - {".5GiB", 512 * 1024 * 1024}, - {"1.3", 1}, - } - for _, c := range testCases { - args := []string{"--max-sql-memory", c.value} - if err := f.Parse(args); err != nil { - t.Fatal(err) - } - if c.expected != serverCfg.MemoryPoolSize { - t.Errorf("expected %d, but got %d", c.expected, serverCfg.MemoryPoolSize) - } - } + {flag: "--max-sql-memory", config: &serverCfg.MemoryPoolSize}, + {flag: "--max-tsdb-memory", config: &serverCfg.TimeSeriesServerConfig.QueryMemoryMax}, + } { + t.Run(tc.flag, func(t *testing.T) { + // Avoid leaking configuration changes after the test ends. + defer initCLIDefaults() + + f := startCmd.Flags() + + // Check absolute values. + testCases := []struct { + value string + expected int64 + }{ + {"100MB", 100 * 1000 * 1000}, + {".5GiB", 512 * 1024 * 1024}, + {"1.3", 1}, + } + for _, c := range testCases { + args := []string{tc.flag, c.value} + if err := f.Parse(args); err != nil { + t.Fatal(err) + } + if c.expected != *tc.config { + t.Errorf("expected %d, but got %d", c.expected, tc.config) + } + } - for _, c := range []string{".30", "0.3"} { - args := []string{"--max-sql-memory", c} - if err := f.Parse(args); err != nil { - t.Fatal(err) - } + for _, c := range []string{".30", "0.3"} { + args := []string{tc.flag, c} + if err := f.Parse(args); err != nil { + t.Fatal(err) + } - // Check fractional values. - maxMem, err := status.GetTotalMemory(context.Background()) - if err != nil { - t.Logf("total memory unknown: %v", err) - return - } - expectedLow := (maxMem * 28) / 100 - expectedHigh := (maxMem * 32) / 100 - if serverCfg.MemoryPoolSize < expectedLow || serverCfg.MemoryPoolSize > expectedHigh { - t.Errorf("expected %d-%d, but got %d", expectedLow, expectedHigh, serverCfg.MemoryPoolSize) - } + // Check fractional values. + maxMem, err := status.GetTotalMemory(context.Background()) + if err != nil { + t.Logf("total memory unknown: %v", err) + return + } + expectedLow := (maxMem * 28) / 100 + expectedHigh := (maxMem * 32) / 100 + if *tc.config < expectedLow || *tc.config > expectedHigh { + t.Errorf("expected %d-%d, but got %d", expectedLow, expectedHigh, *tc.config) + } + } + }) } } diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 431ea4fdf499..6b5e6a319ceb 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -185,6 +185,7 @@ func initTraceDir(ctx context.Context, dir string) { var cacheSizeValue = newBytesOrPercentageValue(&serverCfg.CacheSize, memoryPercentResolver) var sqlSizeValue = newBytesOrPercentageValue(&serverCfg.MemoryPoolSize, memoryPercentResolver) var diskTempStorageSizeValue = newBytesOrPercentageValue(nil /* v */, nil /* percentResolver */) +var tsdbSizeValue = newBytesOrPercentageValue(&serverCfg.TimeSeriesServerConfig.QueryMemoryMax, memoryPercentResolver) func initExternalIODir(ctx context.Context, firstStore base.StoreSpec) (string, error) { externalIODir := startCtx.externalIODir @@ -1101,12 +1102,12 @@ func maybeWarnMemorySizes(ctx context.Context) { // Check that the total suggested "max" memory is well below the available memory. if maxMemory, err := status.GetTotalMemory(ctx); err == nil { - requestedMem := serverCfg.CacheSize + serverCfg.MemoryPoolSize + requestedMem := serverCfg.CacheSize + serverCfg.MemoryPoolSize + serverCfg.TimeSeriesServerConfig.QueryMemoryMax maxRecommendedMem := int64(.75 * float64(maxMemory)) if requestedMem > maxRecommendedMem { log.Ops.Shoutf(ctx, severity.WARNING, - "the sum of --max-sql-memory (%s) and --cache (%s) is larger than 75%% of total RAM (%s).\nThis server is running at increased risk of memory-related failures.", - sqlSizeValue, cacheSizeValue, humanizeutil.IBytes(maxRecommendedMem)) + "the sum of --max-sql-memory (%s), --cache (%s), and --max-tsdb-memory (%s) is larger than 75%% of total RAM (%s).\nThis server is running at increased risk of memory-related failures.", + sqlSizeValue, cacheSizeValue, tsdbSizeValue, humanizeutil.IBytes(maxRecommendedMem)) } } } diff --git a/pkg/internal/sqlsmith/setup.go b/pkg/internal/sqlsmith/setup.go index 479b5ab38dab..3f428e804798 100644 --- a/pkg/internal/sqlsmith/setup.go +++ b/pkg/internal/sqlsmith/setup.go @@ -103,9 +103,6 @@ func randTablesN(r *rand.Rand, n int) string { sb.WriteString(stmt.String()) sb.WriteString(";\n") } - - // TODO(mjibson): add random INSERTs. - return sb.String() } diff --git a/pkg/kv/db.go b/pkg/kv/db.go index dddd45dfd496..88fd9b841329 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -851,6 +851,18 @@ func (db *DB) NewTxn(ctx context.Context, debugName string) *Txn { // from recoverable internal errors, and is automatically committed // otherwise. The retryable function should have no side effects which could // cause problems in the event it must be run more than once. +// For example: +// err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { +// if kv, err := txn.Get(ctx, key); err != nil { +// return err +// } +// // ... +// return nil +// }) +// Note that once the transaction encounters a retryable error, the txn object +// is marked as poisoned and all future ops fail fast until the retry. The +// callback may return either nil or the retryable error. Txn is responsible for +// resetting the transaction and retrying the callback. func (db *DB) Txn(ctx context.Context, retryable func(context.Context, *Txn) error) error { // TODO(radu): we should open a tracing Span here (we need to figure out how // to use the correct tracer). diff --git a/pkg/kv/db_test.go b/pkg/kv/db_test.go index ea9d845f3497..b335dcab003c 100644 --- a/pkg/kv/db_test.go +++ b/pkg/kv/db_test.go @@ -13,6 +13,7 @@ package kv_test import ( "bytes" "context" + "fmt" "testing" "time" @@ -715,3 +716,84 @@ func TestGenerateForcedRetryableError(t *testing.T) { require.True(t, errors.As(err, &retryErr)) require.Equal(t, 1, int(retryErr.Transaction.Epoch)) } + +// Get a retryable error within a db.Txn transaction and verify the retry +// succeeds. We are verifying the behavior is the same whether the retryable +// callback returns the retryable error or returns nil. Both implementations are +// legal - returning early (with either nil or the error) after a retryable +// error is optional. +func TestDB_TxnRetry(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + s, db := setup(t) + defer s.Stopper().Stop(context.Background()) + + testutils.RunTrueAndFalse(t, "returnNil", func(t *testing.T, returnNil bool) { + keyA := fmt.Sprintf("a_return_nil_%t", returnNil) + keyB := fmt.Sprintf("b_return_nil_%t", returnNil) + runNumber := 0 + err := db.Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { + require.NoError(t, txn.Put(ctx, keyA, "1")) + require.NoError(t, txn.Put(ctx, keyB, "1")) + + { + // High priority txn - will abort the other txn. + hpTxn := kv.NewTxn(ctx, db, 0) + require.NoError(t, hpTxn.SetUserPriority(roachpb.MaxUserPriority)) + // Only write if we have not written before, because otherwise we will keep aborting + // the other txn forever. + r, err := hpTxn.Get(ctx, keyA) + require.NoError(t, err) + if !r.Exists() { + require.Zero(t, runNumber) + require.NoError(t, hpTxn.Put(ctx, keyA, "hp txn")) + require.NoError(t, hpTxn.Commit(ctx)) + } else { + // We already wrote to keyA, meaning this is a retry, no need to write again. + require.Equal(t, 1, runNumber) + require.NoError(t, hpTxn.Rollback(ctx)) + } + } + + // Read, so that we'll get a retryable error. + r, err := txn.Get(ctx, keyA) + if runNumber == 0 { + // First run, we should get a retryable error. + require.Zero(t, runNumber) + require.IsType(t, &roachpb.TransactionRetryWithProtoRefreshError{}, err) + require.Equal(t, []byte(nil), r.ValueBytes()) + + // At this point txn is poisoned, and any op returns the same (poisoning) error. + r, err = txn.Get(ctx, keyB) + require.IsType(t, &roachpb.TransactionRetryWithProtoRefreshError{}, err) + require.Equal(t, []byte(nil), r.ValueBytes()) + } else { + // The retry should succeed. + require.Equal(t, 1, runNumber) + require.NoError(t, err) + require.Equal(t, []byte("1"), r.ValueBytes()) + } + runNumber++ + + if returnNil { + return nil + } + // Return the retryable error. + return err + }) + require.NoError(t, err) + require.Equal(t, 2, runNumber) + + err1 := db.Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { + // The high priority txn was overwritten by the successful retry. + kv, e1 := txn.Get(ctx, keyA) + require.NoError(t, e1) + require.Equal(t, []byte("1"), kv.ValueBytes()) + kv, e2 := txn.Get(ctx, keyB) + require.NoError(t, e2) + require.Equal(t, []byte("1"), kv.ValueBytes()) + return nil + }) + require.NoError(t, err1) + }) +} diff --git a/pkg/kv/kvclient/kvcoord/testdata/savepoints b/pkg/kv/kvclient/kvcoord/testdata/savepoints index 55c076058e71..37d572c9e3b8 100644 --- a/pkg/kv/kvclient/kvcoord/testdata/savepoints +++ b/pkg/kv/kvclient/kvcoord/testdata/savepoints @@ -486,6 +486,11 @@ savepoint x abort ---- (*roachpb.TransactionRetryWithProtoRefreshError) +txn id not changed + +reset +---- +txn error cleared txn id changed release x diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go index 23a2c5da206f..4d94f507805a 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go @@ -50,6 +50,12 @@ const ( // txnPending is the normal state for ongoing transactions. txnPending txnState = iota + // txnRetryableError means that the transaction encountered a + // TransactionRetryWithProtoRefreshError, and calls to Send() fail in this + // state. It is possible to move back to txnPending by calling + // ClearTxnRetryableErr(). + txnRetryableError + // txnError means that a batch encountered a non-retriable error. Further // batches except EndTxn(commit=false) will be rejected. txnError @@ -105,6 +111,11 @@ type TxnCoordSender struct { syncutil.Mutex txnState txnState + + // storedRetryableErr is set when txnState == txnRetryableError. This + // storedRetryableErr is returned to clients on Send(). + storedRetryableErr *roachpb.TransactionRetryWithProtoRefreshError + // storedErr is set when txnState == txnError. This storedErr is returned to // clients on Send(). storedErr *roachpb.Error @@ -686,6 +697,8 @@ func (tc *TxnCoordSender) maybeRejectClientLocked( switch tc.mu.txnState { case txnPending: // All good. + case txnRetryableError: + return roachpb.NewError(tc.mu.storedRetryableErr) case txnError: return tc.mu.storedErr case txnFinalized: @@ -712,19 +725,10 @@ func (tc *TxnCoordSender) maybeRejectClientLocked( // The transaction heartbeat observed an aborted transaction record and // this was not due to a synchronous transaction commit and transaction // record garbage collection. - // See the comment on txnHeartbeater.mu.finalizedStatus for more details. + // See the comment on txnHeartbeater.mu.finalObservedStatus for more details. abortedErr := roachpb.NewErrorWithTxn( roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_CLIENT_REJECT), &tc.mu.txn) - if tc.typ == kv.LeafTxn { - // Leaf txns return raw retriable errors (which get handled by the - // root) rather than TransactionRetryWithProtoRefreshError. - return abortedErr - } - // Root txns handle retriable errors. - newTxn := roachpb.PrepareTransactionForRetry( - ctx, abortedErr, roachpb.NormalUserPriority, tc.clock) - return roachpb.NewError(roachpb.NewTransactionRetryWithProtoRefreshError( - abortedErr.String(), tc.mu.txn.ID, newTxn)) + return roachpb.NewError(tc.handleRetryableErrLocked(ctx, abortedErr)) case protoStatus != roachpb.PENDING || hbObservedStatus != roachpb.PENDING: // The transaction proto is in an unexpected state. return roachpb.NewErrorf( @@ -816,6 +820,11 @@ func (tc *TxnCoordSender) handleRetryableErrLocked( errTxnID, // the id of the transaction that encountered the error newTxn) + // Move to a retryable error state, where all Send() calls fail until the + // state is cleared. + tc.mu.txnState = txnRetryableError + tc.mu.storedRetryableErr = retErr + // If the ID changed, it means we had to start a new transaction and the // old one is toast. This TxnCoordSender cannot be used any more - future // Send() calls will be rejected; the client is supposed to create a new @@ -1360,3 +1369,25 @@ func (tc *TxnCoordSender) DeferCommitWait(ctx context.Context) func(context.Cont return tc.maybeCommitWait(ctx, true /* deferred */) } } + +// GetTxnRetryableErr is part of the TxnSender interface. +func (tc *TxnCoordSender) GetTxnRetryableErr( + ctx context.Context, +) *roachpb.TransactionRetryWithProtoRefreshError { + tc.mu.Lock() + defer tc.mu.Unlock() + if tc.mu.txnState == txnRetryableError { + return tc.mu.storedRetryableErr + } + return nil +} + +// ClearTxnRetryableErr is part of the TxnSender interface. +func (tc *TxnCoordSender) ClearTxnRetryableErr(ctx context.Context) { + tc.mu.Lock() + defer tc.mu.Unlock() + if tc.mu.txnState == txnRetryableError { + tc.mu.storedRetryableErr = nil + tc.mu.txnState = txnPending + } +} diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints.go index 5191a94f73a4..8c54c5a9b7d8 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints.go @@ -126,16 +126,12 @@ func (tc *TxnCoordSender) RollbackToSavepoint(ctx context.Context, s kv.Savepoin err = roachpb.NewTransactionRetryWithProtoRefreshError( "cannot rollback to savepoint after a transaction restart", tc.mu.txn.ID, - // The transaction inside this error doesn't matter. - roachpb.Transaction{}, + tc.mu.txn, ) } return err } - // Restore the transaction's state, in case we're rewiding after an error. - tc.mu.txnState = txnPending - tc.mu.active = sp.active for _, reqInt := range tc.interceptorStack { @@ -173,8 +169,7 @@ func (tc *TxnCoordSender) ReleaseSavepoint(ctx context.Context, s kv.SavepointTo err = roachpb.NewTransactionRetryWithProtoRefreshError( "cannot release savepoint after a transaction restart", tc.mu.txn.ID, - // The transaction inside this error doesn't matter. - roachpb.Transaction{}, + tc.mu.txn, ) } return err diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go index dd41d50adf7f..009c1ba4cd5b 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go @@ -124,6 +124,16 @@ func TestSavepoints(t *testing.T) { } fmt.Fprintf(&buf, "txn id %s\n", changed) + case "reset": + prevID := txn.ID() + txn.PrepareForRetry(ctx) + changed := "changed" + if prevID == txn.ID() { + changed = "not changed" + } + fmt.Fprintf(&buf, "txn error cleared\n") + fmt.Fprintf(&buf, "txn id %s\n", changed) + case "put": b := txn.NewBatch() b.Put(td.CmdArgs[0].Key, td.CmdArgs[1].Key) diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go index 32860643b62a..27ada82dfd11 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go @@ -254,6 +254,85 @@ func TestTxnCoordSenderHeartbeat(t *testing.T) { } } +// Verify the txn sees a retryable error without using the handle: Normally the +// caller uses the txn handle directly, and if there is a retryable error then +// Send() fails and the handle gets "poisoned", meaning, the coordinator will be +// in state txnRetryableError instead of txnPending. But sometimes the handle +// can be idle and aborted by a heartbeat failure. This test verifies that in +// those cases the state of the handle ends up as txnRetryableError. +// This is important to verify because if the handle stays in txnPending then +// GetTxnRetryableErr() returns nil, and PrepareForRetry() will not reset the +// handle. +func TestDB_PrepareForRetryAfterHeartbeatFailure(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Create a DB with a short heartbeat interval. + s := createTestDB(t) + defer s.Stop() + ctx := context.Background() + ambient := s.AmbientCtx + tsf := kvcoord.NewTxnCoordSenderFactory( + kvcoord.TxnCoordSenderFactoryConfig{ + AmbientCtx: ambient, + // Short heartbeat interval. + HeartbeatInterval: time.Millisecond, + Settings: s.Cfg.Settings, + Clock: s.Clock, + Stopper: s.Stopper(), + }, + kvcoord.NewDistSenderForLocalTestCluster( + ctx, + s.Cfg.Settings, &roachpb.NodeDescriptor{NodeID: 1}, + ambient.Tracer, s.Clock, s.Latency, s.Stores, s.Stopper(), s.Gossip, + ), + ) + db := kv.NewDB(ambient, tsf, s.Clock, s.Stopper()) + + // Create a txn which will be aborted by a high priority txn. + txn := kv.NewTxn(ctx, db, 0) + + // We first write to one range, then a high priority txn will abort our txn, + // then we will try to read from another range until we see that the + // transaction is poisoned because of a heartbeat failure. + // Note that if we read from the same range then we will check the AbortSpan + // and fail immediately (Send() will fail), even before the heartbeat failure, + // which is not the case we want to test here. + keyA := roachpb.Key("a") + keyC := roachpb.Key("c") + splitKey := roachpb.Key("b") + require.NoError(t, s.DB.AdminSplit(ctx, splitKey, hlc.MaxTimestamp)) + require.NoError(t, txn.Put(ctx, keyA, "1")) + + { + // High priority txn - will abort the other txn. + hpTxn := kv.NewTxn(ctx, db, 0) + require.NoError(t, hpTxn.SetUserPriority(roachpb.MaxUserPriority)) + require.NoError(t, hpTxn.Put(ctx, keyA, "hp txn")) + require.NoError(t, hpTxn.Commit(ctx)) + } + + tc := txn.Sender().(*kvcoord.TxnCoordSender) + + // Wait until we know that the handle was poisoned due to a heartbeat failure. + testutils.SucceedsSoon(t, func() error { + _, err := txn.Get(ctx, keyC) + if err == nil { + return errors.New("the handle is not poisoned yet") + } + require.IsType(t, &roachpb.TransactionRetryWithProtoRefreshError{}, err) + return nil + }) + + // At this point the handle should be in state txnRetryableError - verify we + // can read the error. + pErr := tc.GetTxnRetryableErr(ctx) + require.NotNil(t, pErr) + require.Equal(t, txn.ID(), pErr.TxnID) + // The transaction was aborted, therefore we should have a new transaction ID. + require.NotEqual(t, pErr.TxnID, pErr.Transaction.ID) +} + // getTxn fetches the requested key and returns the transaction info. func getTxn(ctx context.Context, txn *kv.Txn) (*roachpb.Transaction, *roachpb.Error) { txnMeta := txn.TestingCloneTxn().TxnMeta @@ -648,6 +727,7 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { // The test's name. name string pErrGen func(txn *roachpb.Transaction) *roachpb.Error + callPrepareForRetry bool expEpoch enginepb.TxnEpoch expPri enginepb.TxnPriority expWriteTS, expReadTS hlc.Timestamp @@ -705,18 +785,32 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { expReadTS: plus20, }, { - // On abort, nothing changes but we get a new priority to use for - // the next attempt. + // On abort, nothing changes - we are left with a poisoned txn (unless we + // call PrepareForRetry as in the next test case). name: "TransactionAbortedError", pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { txn.WriteTimestamp = plus20 txn.Priority = 10 return roachpb.NewErrorWithTxn(&roachpb.TransactionAbortedError{}, txn) }, - expNewTransaction: true, - expPri: 10, - expWriteTS: plus20, - expReadTS: plus20, + expPri: 1, + expWriteTS: origTS, + expReadTS: origTS, + }, + { + // On abort, reset the txn by calling PrepareForRetry, and then we get a + // new priority to use for the next attempt. + name: "TransactionAbortedError with PrepareForRetry", + pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { + txn.WriteTimestamp = plus20 + txn.Priority = 10 + return roachpb.NewErrorWithTxn(&roachpb.TransactionAbortedError{}, txn) + }, + callPrepareForRetry: true, + expNewTransaction: true, + expPri: 10, + expWriteTS: plus20, + expReadTS: plus20, }, { // On failed push, new epoch begins just past the pushed timestamp. @@ -806,6 +900,9 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { err := txn.Put(ctx, key, []byte("value")) stopper.Stop(ctx) + if test.callPrepareForRetry { + txn.PrepareForRetry(ctx) + } if test.name != "nil" && err == nil { t.Fatalf("expected an error") } @@ -1931,9 +2028,9 @@ func TestEndWriteRestartReadOnlyTransaction(t *testing.T) { calls = nil firstIter := true if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + var err error if firstIter { firstIter = false - var err error if write { err = txn.Put(ctx, "consider", "phlebas") } else /* locking read */ { @@ -1946,7 +2043,7 @@ func TestEndWriteRestartReadOnlyTransaction(t *testing.T) { if !success { return errors.New("aborting on purpose") } - return nil + return err }); err == nil != success { t.Fatalf("expected error: %t, got error: %v", !success, err) } diff --git a/pkg/kv/kvclient/kvcoord/txnstate_string.go b/pkg/kv/kvclient/kvcoord/txnstate_string.go index 0189f4fdf40d..23ab14da8ad2 100644 --- a/pkg/kv/kvclient/kvcoord/txnstate_string.go +++ b/pkg/kv/kvclient/kvcoord/txnstate_string.go @@ -9,13 +9,14 @@ func _() { // Re-run the stringer command to generate them again. var x [1]struct{} _ = x[txnPending-0] - _ = x[txnError-1] - _ = x[txnFinalized-2] + _ = x[txnRetryableError-1] + _ = x[txnError-2] + _ = x[txnFinalized-3] } -const _txnState_name = "txnPendingtxnErrortxnFinalized" +const _txnState_name = "txnPendingtxnRetryableErrortxnErrortxnFinalized" -var _txnState_index = [...]uint8{0, 10, 18, 30} +var _txnState_index = [...]uint8{0, 10, 27, 35, 47} func (i txnState) String() string { if i < 0 || i >= txnState(len(_txnState_index)-1) { diff --git a/pkg/kv/mock_transactional_sender.go b/pkg/kv/mock_transactional_sender.go index e56c1fc1a684..a793bd24a044 100644 --- a/pkg/kv/mock_transactional_sender.go +++ b/pkg/kv/mock_transactional_sender.go @@ -217,6 +217,17 @@ func (m *MockTransactionalSender) DeferCommitWait(ctx context.Context) func(cont panic("unimplemented") } +// GetTxnRetryableErr is part of the TxnSender interface. +func (m *MockTransactionalSender) GetTxnRetryableErr( + ctx context.Context, +) *roachpb.TransactionRetryWithProtoRefreshError { + return nil +} + +// ClearTxnRetryableErr is part of the TxnSender interface. +func (m *MockTransactionalSender) ClearTxnRetryableErr(ctx context.Context) { +} + // MockTxnSenderFactory is a TxnSenderFactory producing MockTxnSenders. type MockTxnSenderFactory struct { senderFunc func(context.Context, *roachpb.Transaction, roachpb.BatchRequest) ( diff --git a/pkg/kv/sender.go b/pkg/kv/sender.go index a98bb744ffb3..dc8e8c2de390 100644 --- a/pkg/kv/sender.go +++ b/pkg/kv/sender.go @@ -324,6 +324,15 @@ type TxnSender interface { // violations where a future, causally dependent transaction may fail to // observe the writes performed by this transaction. DeferCommitWait(ctx context.Context) func(context.Context) error + + // GetTxnRetryableErr returns an error if the TxnSender had a retryable error, + // otherwise nil. In this state Send() always fails with the same retryable + // error. ClearTxnRetryableErr can be called to clear this error and make + // TxnSender usable again. + GetTxnRetryableErr(ctx context.Context) *roachpb.TransactionRetryWithProtoRefreshError + + // ClearTxnRetryableErr clears the retryable error, if any. + ClearTxnRetryableErr(ctx context.Context) } // SteppingMode is the argument type to ConfigureStepping. diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 4c9d80c49da1..0254aa95b1c3 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -1012,24 +1012,32 @@ func (txn *Txn) exec(ctx context.Context, fn func(context.Context, *Txn) error) break } - txn.PrepareForRetry(ctx, err) + txn.PrepareForRetry(ctx) } return err } -// PrepareForRetry needs to be called before an retry to perform some -// book-keeping. -// -// TODO(andrei): I think this is called in the wrong place. See #18170. -func (txn *Txn) PrepareForRetry(ctx context.Context, err error) { +// PrepareForRetry needs to be called before a retry to perform some +// book-keeping and clear errors when possible. +func (txn *Txn) PrepareForRetry(ctx context.Context) { + // TODO(andrei): I think commit triggers are reset in the wrong place. See #18170. + txn.commitTriggers = nil + + txn.mu.Lock() + defer txn.mu.Unlock() + + retryErr := txn.mu.sender.GetTxnRetryableErr(ctx) + if retryErr == nil { + return + } if txn.typ != RootTxn { - panic(errors.WithContextTags(errors.NewAssertionErrorWithWrappedErrf(err, "PrepareForRetry() called on leaf txn"), ctx)) + panic(errors.WithContextTags(errors.NewAssertionErrorWithWrappedErrf( + retryErr, "PrepareForRetry() called on leaf txn"), ctx)) } - - txn.commitTriggers = nil - log.VEventf(ctx, 2, "automatically retrying transaction: %s because of error: %s", - txn.DebugName(), err) + log.VEventf(ctx, 2, "retrying transaction: %s because of a retryable error: %s", + txn.debugNameLocked(), retryErr) + txn.handleRetryableErrLocked(ctx, retryErr) } // IsRetryableErrMeantForTxn returns true if err is a retryable @@ -1105,21 +1113,13 @@ func (txn *Txn) Send( "requestTxnID: %s, retryErr.TxnID: %s. retryErr: %s", requestTxnID, retryErr.TxnID, retryErr) } - if txn.typ == RootTxn { - // On root senders, we bump the sender's identity upon retry errors. - txn.mu.Lock() - txn.handleErrIfRetryableLocked(ctx, retryErr) - txn.mu.Unlock() - } } return br, pErr } -func (txn *Txn) handleErrIfRetryableLocked(ctx context.Context, err error) { - var retryErr *roachpb.TransactionRetryWithProtoRefreshError - if !errors.As(err, &retryErr) { - return - } +func (txn *Txn) handleRetryableErrLocked( + ctx context.Context, retryErr *roachpb.TransactionRetryWithProtoRefreshError, +) { txn.resetDeadlineLocked() txn.replaceRootSenderIfTxnAbortedLocked(ctx, retryErr, retryErr.TxnID) } @@ -1311,7 +1311,10 @@ func (txn *Txn) GetLeafTxnInputStateOrRejectClient( defer txn.mu.Unlock() tfs, err := txn.mu.sender.GetLeafTxnInputState(ctx, OnlyPending) if err != nil { - txn.handleErrIfRetryableLocked(ctx, err) + var retryErr *roachpb.TransactionRetryWithProtoRefreshError + if errors.As(err, &retryErr) { + txn.handleRetryableErrLocked(ctx, retryErr) + } return nil, err } return tfs, nil @@ -1405,8 +1408,9 @@ func (txn *Txn) replaceRootSenderIfTxnAbortedLocked( return } if !retryErr.PrevTxnAborted() { - // We don't need a new transaction as a result of this error. Nothing more - // to do. + // We don't need a new transaction as a result of this error, but we may + // have a retryable error that should be cleared. + txn.mu.sender.ClearTxnRetryableErr(ctx) return } diff --git a/pkg/server/server.go b/pkg/server/server.go index 201dea9d4021..5359507b8bf9 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -436,13 +436,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { cfg.AmbientCtx, st, nodeDialer, grpcServer.Server, stopper, ) - tsDB := ts.NewDB(db, cfg.Settings) - registry.AddMetricStruct(tsDB.Metrics()) - nodeCountFn := func() int64 { - return nodeLiveness.Metrics().LiveNodes.Value() - } - sTS := ts.MakeServer(cfg.AmbientCtx, tsDB, nodeCountFn, cfg.TimeSeriesServerConfig, stopper) - ctSender := sidetransport.NewSender(stopper, st, clock, nodeDialer) stores := kvserver.NewStores(cfg.AmbientCtx, clock) ctReceiver := sidetransport.NewReceiver(nodeIDContainer, stopper, stores, nil /* testingKnobs */) @@ -493,6 +486,16 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { kvMemoryMonitor.Stop(ctx) })) + tsDB := ts.NewDB(db, cfg.Settings) + registry.AddMetricStruct(tsDB.Metrics()) + nodeCountFn := func() int64 { + return nodeLiveness.Metrics().LiveNodes.Value() + } + sTS := ts.MakeServer( + cfg.AmbientCtx, tsDB, nodeCountFn, cfg.TimeSeriesServerConfig, + sqlMonitorAndMetrics.rootSQLMemoryMonitor, stopper, + ) + storeCfg := kvserver.StoreConfig{ DefaultSpanConfig: cfg.DefaultZoneConfig.AsSpanConfig(), Settings: st, diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index d96b15caf3b4..a0a86e80865e 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -763,13 +763,7 @@ func (ex *connExecutor) execStmtInOpenState( } txn.ManualRestart(ctx, ex.server.cfg.Clock.Now()) payload := eventRetriableErrPayload{ - err: roachpb.NewTransactionRetryWithProtoRefreshError( - "serializable transaction timestamp pushed (detected by connExecutor)", - txn.ID(), - // No updated transaction required; we've already manually updated our - // client.Txn. - roachpb.Transaction{}, - ), + err: txn.PrepareRetryableError(ctx, "serializable transaction timestamp pushed (detected by connExecutor)"), rewCap: rc, } return ev, payload, nil diff --git a/pkg/sql/conn_fsm.go b/pkg/sql/conn_fsm.go index 26521abd0265..ba0c2bc820fc 100644 --- a/pkg/sql/conn_fsm.go +++ b/pkg/sql/conn_fsm.go @@ -277,22 +277,10 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ stateOpen{ImplicitTxn: fsm.Var("implicitTxn")}: { // This is the case where we auto-retry. eventRetriableErr{CanAutoRetry: fsm.True, IsCommit: fsm.Any}: { - // We leave the transaction in Open. In particular, we don't move to - // RestartWait, as there'd be nothing to move us back from RestartWait to - // Open. - // Note: Preparing the KV txn for restart has already happened by this - // point. + // Rewind and auto-retry - the transaction should stay in the Open state. Description: "Retriable err; will auto-retry", Next: stateOpen{ImplicitTxn: fsm.Var("implicitTxn")}, - Action: func(args fsm.Args) error { - // The caller will call rewCap.rewindAndUnlock(). - args.Extended.(*txnState).setAdvanceInfo( - rewind, - args.Payload.(eventRetriableErrPayload).rewCap, - txnEvent{eventType: txnRestart}, - ) - return nil - }, + Action: prepareTxnForRetryWithRewind, }, }, // Handle the errors in implicit txns. They move us to NoTxn. @@ -321,20 +309,11 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ eventTxnRestart{}: { Description: "ROLLBACK TO SAVEPOINT cockroach_restart", Next: stateOpen{ImplicitTxn: fsm.False}, - Action: func(args fsm.Args) error { - args.Extended.(*txnState).setAdvanceInfo( - advanceOne, - noRewind, - txnEvent{eventType: txnRestart}, - ) - return nil - }, + Action: prepareTxnForRetry, }, eventRetriableErr{CanAutoRetry: fsm.False, IsCommit: fsm.False}: { Next: stateAborted{}, Action: func(args fsm.Args) error { - // Note: Preparing the KV txn for restart has already happened by this - // point. args.Extended.(*txnState).setAdvanceInfo( skipBatch, noRewind, @@ -429,14 +408,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{ eventTxnRestart{}: { Description: "ROLLBACK TO SAVEPOINT cockroach_restart", Next: stateOpen{ImplicitTxn: fsm.False}, - Action: func(args fsm.Args) error { - args.Extended.(*txnState).setAdvanceInfo( - advanceOne, - noRewind, - txnEvent{eventType: txnRestart}, - ) - return nil - }, + Action: prepareTxnForRetry, }, }, @@ -516,7 +488,9 @@ func (ts *txnState) finishTxn(ev txnEventType) error { // cleanupAndFinishOnError rolls back the KV txn and finishes the SQL txn. func cleanupAndFinishOnError(args fsm.Args) error { ts := args.Extended.(*txnState) + ts.mu.Lock() ts.mu.txn.CleanupOnError(ts.Ctx, args.Payload.(payloadWithError).errorCause()) + ts.mu.Unlock() finishedTxnID := ts.finishSQLTxn() ts.setAdvanceInfo( skipBatch, @@ -526,6 +500,33 @@ func cleanupAndFinishOnError(args fsm.Args) error { return nil } +func prepareTxnForRetry(args fsm.Args) error { + ts := args.Extended.(*txnState) + ts.mu.Lock() + ts.mu.txn.PrepareForRetry(ts.Ctx) + ts.mu.Unlock() + ts.setAdvanceInfo( + advanceOne, + noRewind, + txnEvent{eventType: txnRestart}, + ) + return nil +} + +func prepareTxnForRetryWithRewind(args fsm.Args) error { + ts := args.Extended.(*txnState) + ts.mu.Lock() + ts.mu.txn.PrepareForRetry(ts.Ctx) + ts.mu.Unlock() + // The caller will call rewCap.rewindAndUnlock(). + ts.setAdvanceInfo( + rewind, + args.Payload.(eventRetriableErrPayload).rewCap, + txnEvent{eventType: txnRestart}, + ) + return nil +} + // BoundTxnStateTransitions is the state machine used by the InternalExecutor // when running SQL inside a higher-level txn. It's a very limited state // machine: it doesn't allow starting or finishing txns, auto-retries, etc. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/join b/pkg/sql/opt/exec/execbuilder/testdata/join index c7a95bb37df0..fb5c514209fd 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/join +++ b/pkg/sql/opt/exec/execbuilder/testdata/join @@ -431,120 +431,115 @@ vectorized: true │ render relname: relname │ └── • hash join (inner) - │ columns: (attrelid, attname, attnum, attrelid, attname, attnum, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname, generate_series, oid, relname, relnamespace, oid, nspname, objid, refobjid, oid, relname, relkind) + │ columns: (oid, nspname, oid, relname, relnamespace, attrelid, attname, attnum, attrelid, attname, attnum, objid, refobjid, oid, relname, relkind, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname, generate_series) │ estimated row count: 110,908 (missing stats) - │ equality: (oid) = (objid) + │ equality: (oid) = (attrelid) │ ├── • hash join (inner) - │ │ columns: (attrelid, attname, attnum, attrelid, attname, attnum, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname, generate_series, oid, relname, relnamespace, oid, nspname) - │ │ estimated row count: 114,302 (missing stats) - │ │ equality: (relnamespace) = (oid) + │ │ columns: (oid, nspname, oid, relname, relnamespace) + │ │ estimated row count: 9,801 (missing stats) + │ │ equality: (oid) = (relnamespace) │ │ - │ ├── • hash join (inner) - │ │ │ columns: (attrelid, attname, attnum, attrelid, attname, attnum, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname, generate_series, oid, relname, relnamespace) - │ │ │ estimated row count: 11,557 (missing stats) - │ │ │ equality: (attrelid) = (oid) - │ │ │ pred: attnum = confkey[generate_series] - │ │ │ - │ │ ├── • hash join (inner) - │ │ │ │ columns: (attrelid, attname, attnum, attrelid, attname, attnum, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname, generate_series) - │ │ │ │ estimated row count: 3,502 (missing stats) - │ │ │ │ equality: (attrelid) = (confrelid) - │ │ │ │ - │ │ │ ├── • virtual table - │ │ │ │ columns: (attrelid, attname, attnum) - │ │ │ │ estimated row count: 1,000 (missing stats) - │ │ │ │ table: pg_attribute@primary - │ │ │ │ - │ │ │ └── • cross join (inner) - │ │ │ │ columns: (attrelid, attname, attnum, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname, generate_series) - │ │ │ │ estimated row count: 354 (missing stats) - │ │ │ │ pred: attnum = conkey[generate_series] - │ │ │ │ - │ │ │ ├── • hash join (inner) - │ │ │ │ │ columns: (attrelid, attname, attnum, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname) - │ │ │ │ │ estimated row count: 107 (missing stats) - │ │ │ │ │ equality: (relnamespace) = (oid) - │ │ │ │ │ - │ │ │ │ ├── • merge join (inner) - │ │ │ │ │ │ columns: (attrelid, attname, attnum, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace) - │ │ │ │ │ │ estimated row count: 105 (missing stats) - │ │ │ │ │ │ equality: (attrelid) = (oid) - │ │ │ │ │ │ merge ordering: +"(attrelid=oid)" - │ │ │ │ │ │ - │ │ │ │ │ ├── • virtual table - │ │ │ │ │ │ columns: (attrelid, attname, attnum) - │ │ │ │ │ │ ordering: +attrelid - │ │ │ │ │ │ estimated row count: 1,000 (missing stats) - │ │ │ │ │ │ table: pg_attribute@pg_attribute_attrelid_idx - │ │ │ │ │ │ - │ │ │ │ │ └── • virtual table lookup join (inner) - │ │ │ │ │ │ columns: (oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace) - │ │ │ │ │ │ ordering: +conrelid - │ │ │ │ │ │ estimated row count: 10 (missing stats) - │ │ │ │ │ │ table: pg_class@pg_class_oid_idx - │ │ │ │ │ │ equality: (conrelid) = (oid) - │ │ │ │ │ │ pred: relname = 'orders' - │ │ │ │ │ │ - │ │ │ │ │ └── • filter - │ │ │ │ │ │ columns: (oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey) - │ │ │ │ │ │ ordering: +conrelid - │ │ │ │ │ │ estimated row count: 10 (missing stats) - │ │ │ │ │ │ filter: contype = 'f' - │ │ │ │ │ │ - │ │ │ │ │ └── • virtual table - │ │ │ │ │ columns: (oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey) - │ │ │ │ │ ordering: +conrelid - │ │ │ │ │ estimated row count: 1,000 (missing stats) - │ │ │ │ │ table: pg_constraint@pg_constraint_conrelid_idx - │ │ │ │ │ - │ │ │ │ └── • filter - │ │ │ │ │ columns: (oid, nspname) - │ │ │ │ │ estimated row count: 10 (missing stats) - │ │ │ │ │ filter: nspname = 'public' - │ │ │ │ │ - │ │ │ │ └── • virtual table - │ │ │ │ columns: (oid, nspname) - │ │ │ │ estimated row count: 1,000 (missing stats) - │ │ │ │ table: pg_namespace@primary - │ │ │ │ - │ │ │ └── • project set - │ │ │ │ columns: (generate_series) - │ │ │ │ estimated row count: 10 - │ │ │ │ render 0: generate_series(1, 32) - │ │ │ │ - │ │ │ └── • emptyrow - │ │ │ columns: () - │ │ │ - │ │ └── • virtual table - │ │ columns: (oid, relname, relnamespace) - │ │ estimated row count: 1,000 (missing stats) - │ │ table: pg_class@primary + │ ├── • virtual table + │ │ columns: (oid, nspname) + │ │ estimated row count: 1,000 (missing stats) + │ │ table: pg_namespace@primary │ │ │ └── • virtual table - │ columns: (oid, nspname) + │ columns: (oid, relname, relnamespace) │ estimated row count: 1,000 (missing stats) - │ table: pg_namespace@primary + │ table: pg_class@primary │ └── • hash join (inner) - │ columns: (objid, refobjid, oid, relname, relkind) - │ estimated row count: 99 (missing stats) - │ equality: (refobjid) = (oid) + │ columns: (attrelid, attname, attnum, attrelid, attname, attnum, objid, refobjid, oid, relname, relkind, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname, generate_series) + │ estimated row count: 1,779 (missing stats) + │ equality: (attrelid) = (oid) + │ pred: attnum = conkey[generate_series] │ ├── • virtual table - │ columns: (objid, refobjid) + │ columns: (attrelid, attname, attnum) │ estimated row count: 1,000 (missing stats) - │ table: pg_depend@primary + │ table: pg_attribute@primary │ - └── • filter - │ columns: (oid, relname, relkind) - │ estimated row count: 10 (missing stats) - │ filter: relkind = 'i' + └── • cross join (inner) + │ columns: (attrelid, attname, attnum, objid, refobjid, oid, relname, relkind, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname, generate_series) + │ estimated row count: 539 (missing stats) + │ pred: attnum = confkey[generate_series] │ - └── • virtual table - columns: (oid, relname, relkind) - estimated row count: 1,000 (missing stats) - table: pg_class@primary + ├── • hash join (inner) + │ │ columns: (attrelid, attname, attnum, objid, refobjid, oid, relname, relkind, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname) + │ │ estimated row count: 163 (missing stats) + │ │ equality: (attrelid) = (confrelid) + │ │ + │ ├── • virtual table + │ │ columns: (attrelid, attname, attnum) + │ │ estimated row count: 1,000 (missing stats) + │ │ table: pg_attribute@primary + │ │ + │ └── • hash join (inner) + │ │ columns: (objid, refobjid, oid, relname, relkind, oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname) + │ │ estimated row count: 17 (missing stats) + │ │ equality: (objid) = (oid) + │ │ + │ ├── • hash join (inner) + │ │ │ columns: (objid, refobjid, oid, relname, relkind) + │ │ │ estimated row count: 99 (missing stats) + │ │ │ equality: (refobjid) = (oid) + │ │ │ + │ │ ├── • virtual table + │ │ │ columns: (objid, refobjid) + │ │ │ estimated row count: 1,000 (missing stats) + │ │ │ table: pg_depend@primary + │ │ │ + │ │ └── • filter + │ │ │ columns: (oid, relname, relkind) + │ │ │ estimated row count: 10 (missing stats) + │ │ │ filter: relkind = 'i' + │ │ │ + │ │ └── • virtual table + │ │ columns: (oid, relname, relkind) + │ │ estimated row count: 1,000 (missing stats) + │ │ table: pg_class@primary + │ │ + │ └── • hash join (inner) + │ │ columns: (oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace, oid, nspname) + │ │ estimated row count: 11 (missing stats) + │ │ equality: (relnamespace) = (oid) + │ │ + │ ├── • virtual table lookup join (inner) + │ │ │ columns: (oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey, oid, relname, relnamespace) + │ │ │ estimated row count: 10 (missing stats) + │ │ │ table: pg_class@pg_class_oid_idx + │ │ │ equality: (conrelid) = (oid) + │ │ │ pred: relname = 'orders' + │ │ │ + │ │ └── • filter + │ │ │ columns: (oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey) + │ │ │ estimated row count: 10 (missing stats) + │ │ │ filter: contype = 'f' + │ │ │ + │ │ └── • virtual table + │ │ columns: (oid, conname, contype, condeferrable, condeferred, conrelid, confrelid, confupdtype, confdeltype, conkey, confkey) + │ │ estimated row count: 1,000 (missing stats) + │ │ table: pg_constraint@primary + │ │ + │ └── • filter + │ │ columns: (oid, nspname) + │ │ estimated row count: 10 (missing stats) + │ │ filter: nspname = 'public' + │ │ + │ └── • virtual table + │ columns: (oid, nspname) + │ estimated row count: 1,000 (missing stats) + │ table: pg_namespace@primary + │ + └── • project set + │ columns: (generate_series) + │ estimated row count: 10 + │ render 0: generate_series(1, 32) + │ + └── • emptyrow + columns: () # Ensure that left joins on non-null foreign keys turn into inner joins statement ok diff --git a/pkg/sql/opt/testutils/opttester/reorder_joins.go b/pkg/sql/opt/testutils/opttester/reorder_joins.go index 8621fb2b14c2..261b4d2d2918 100644 --- a/pkg/sql/opt/testutils/opttester/reorder_joins.go +++ b/pkg/sql/opt/testutils/opttester/reorder_joins.go @@ -11,7 +11,6 @@ package opttester import ( - "bytes" "fmt" "strconv" "strings" @@ -38,10 +37,7 @@ import ( func (ot *OptTester) ReorderJoins() (string, error) { ot.builder.Reset() o := ot.makeOptimizer() - - // Map from the first ColumnID of each base relation to its assigned name. - // this ensures that aliases are consistent across different calls to Reorder. - names := map[opt.ColumnID]string{} + jof := newJoinOrderFormatter(o) // joinsConsidered counts the number of joins which joinOrderBuilder attempts // to add to the memo during each call to Reorder. @@ -53,47 +49,43 @@ func (ot *OptTester) ReorderJoins() (string, error) { func( join memo.RelExpr, vertexes []memo.RelExpr, - edges []memo.FiltersExpr, - edgeOps []opt.Operator, + edges []xform.OnReorderEdgeParam, ) { if treeNum > 1 { // This isn't the first Reorder call. Output the number of joins added to // the memo by the last call to Reorder. - ot.builder.WriteString(fmt.Sprintf("\nJoins Considered: %v\n", joinsConsidered)) + ot.output(fmt.Sprintf("Joins Considered: %v\n", joinsConsidered)) joinsConsidered = 0 } - ot.builder.WriteString(separator("-")) - ot.builder.WriteString(fmt.Sprintf("----Join Tree #%v----\n", treeNum)) - ot.builder.WriteString(o.FormatExpr(join, memo.ExprFmtHideAll)) - ot.builder.WriteString("\n----Vertexes----\n") - ot.builder.WriteString(outputVertexes(vertexes, names, o)) - ot.builder.WriteString("----Edges----\n") + ot.separator("-") + ot.output(fmt.Sprintf("Join Tree #%d\n", treeNum)) + ot.separator("-") + ot.indent(o.FormatExpr(join, memo.ExprFmtHideAll)) + ot.output("Vertexes\n") + for i := range vertexes { + ot.indent(jof.formatVertex(vertexes[i])) + } + ot.output("Edges\n") for i := range edges { - ot.builder.WriteString(outputEdge(edges[i], edgeOps[i], o)) + ot.indent(jof.formatEdge(edges[i])) } - ot.builder.WriteString("\n") treeNum++ relsJoinedLast = "" }) o.JoinOrderBuilder().NotifyOnAddJoin(func(left, right, all, refs []memo.RelExpr, op opt.Operator) { - relsToJoin := outputRels(all, names) + relsToJoin := jof.formatVertexSet(all) if relsToJoin != relsJoinedLast { - ot.builder.WriteString( - fmt.Sprintf( - "----Joining %s----\n", - relsToJoin, - ), - ) + ot.output(fmt.Sprintf("Joining %s\n", relsToJoin)) relsJoinedLast = relsToJoin } - ot.builder.WriteString( + ot.indent( fmt.Sprintf( - "%s %s refs [%s] [%s]\n", - outputRels(left, names), - outputRels(right, names), - outputRels(refs, names), - outputOp(op), + "%s %s [%s, refs=%s]", + jof.formatVertexSet(left), + jof.formatVertexSet(right), + joinOpLabel(op), + jof.formatVertexSet(refs), ), ) joinsConsidered++ @@ -103,61 +95,120 @@ func (ot *OptTester) ReorderJoins() (string, error) { if err != nil { return "", err } - ot.builder.WriteString(fmt.Sprintf("\nJoins Considered: %v\n", joinsConsidered)) - ot.builder.WriteString(separator("-")) - ot.builder.WriteString("----Final Plan----\n") - ot.builder.WriteString(ot.FormatExpr(expr)) - ot.builder.WriteString(separator("-")) + ot.output(fmt.Sprintf("Joins Considered: %d\n", joinsConsidered)) + ot.separator("=") + ot.output("Final Plan\n") + ot.separator("=") + ot.output(ot.FormatExpr(expr)) return ot.builder.String(), err } -// outputVertexes outputs each base relation in the vertexes slice along with +type joinOrderFormatter struct { + o *xform.Optimizer + + // relLabels is a map from the first ColumnID of each base relation to its + // assigned label. + relLabels map[opt.ColumnID]string +} + +// newJoinOrderFormatter returns an initialized joinOrderFormatter. +func newJoinOrderFormatter(o *xform.Optimizer) *joinOrderFormatter { + return &joinOrderFormatter{ + o: o, + relLabels: make(map[opt.ColumnID]string), + } +} + +// formatVertex outputs each base relation in the vertexes slice along with // its alias. -func outputVertexes( - vertexes []memo.RelExpr, names map[opt.ColumnID]string, o *xform.Optimizer, -) string { - buf := bytes.Buffer{} - for i := range vertexes { - firstCol, ok := vertexes[i].Relational().OutputCols.Next(0) - if !ok { - panic(errors.AssertionFailedf("failed to retrieve column from %v", vertexes[i].Op())) - } - name, ok := names[firstCol] - if !ok { - name = getRelationName(len(names)) - names[firstCol] = name - } - buf.WriteString( - fmt.Sprintf( - "%s:\n%s", - name, - o.FormatExpr(vertexes[i], memo.ExprFmtHideAll), - ), - ) - buf.WriteString("\n") +func (jof *joinOrderFormatter) formatVertex(vertex memo.RelExpr) string { + var b strings.Builder + b.WriteString(jof.relLabel(vertex)) + b.WriteString(":\n") + expr := jof.o.FormatExpr(vertex, memo.ExprFmtHideAll) + expr = strings.TrimRight(expr, " \n\t\r") + lines := strings.Split(expr, "\n") + for _, line := range lines { + b.WriteString(fmt.Sprintf(" %s\n", line)) + } + return b.String() +} + +// formatVertexSet outputs each base relation in the vertexes slice along with +// its alias. +func (jof *joinOrderFormatter) formatVertexSet(vertexSet []memo.RelExpr) string { + var b strings.Builder + for i := range vertexSet { + b.WriteString(jof.relLabel(vertexSet[i])) } - return buf.String() + return b.String() } -// outputEdge returns a formatted string for the given FiltersItem along with +// formatEdge returns a formatted string for the given FiltersItem along with // the type of join the edge came from, like so: "x = a left". -func outputEdge(edge memo.FiltersExpr, op opt.Operator, o *xform.Optimizer) string { - buf := bytes.Buffer{} - if len(edge) == 0 { - buf.WriteString("cross") +func (jof *joinOrderFormatter) formatEdge(edge xform.OnReorderEdgeParam) string { + var b strings.Builder + if len(edge.Filters) == 0 { + b.WriteString("cross") } else { - for i := range edge { + for i := range edge.Filters { if i != 0 { - buf.WriteString(", ") + b.WriteString(", ") } - buf.WriteString(strings.TrimSuffix(o.FormatExpr(&edge[i], memo.ExprFmtHideAll), "\n")) + b.WriteString(strings.TrimSuffix(jof.o.FormatExpr(&edge.Filters[i], memo.ExprFmtHideAll), "\n")) + } + } + b.WriteString(fmt.Sprintf( + " [%s, ses=%s, tes=%s, rules=%s]", + joinOpLabel(edge.Op), + jof.formatVertexSet(edge.SES), + jof.formatVertexSet(edge.TES), + jof.formatRules(edge.Rules), + )) + return b.String() +} + +func (jof *joinOrderFormatter) formatRules(rules []xform.OnReorderRuleParam) string { + var b strings.Builder + b.WriteRune('(') + for i, rule := range rules { + if i > 0 { + b.WriteRune(',') } + b.WriteString(fmt.Sprintf( + "%s->%s", + jof.formatVertexSet(rule.From), + jof.formatVertexSet(rule.To), + )) } - buf.WriteString(fmt.Sprintf(" [%s]\n", outputOp(op))) - return buf.String() + b.WriteRune(')') + return b.String() } -func outputOp(op opt.Operator) string { +// relLabel returns the label for the given relation. Labels will follow the +// pattern A, B, ..., Z, A1, B1, etc. +func (jof *joinOrderFormatter) relLabel(e memo.RelExpr) string { + firstCol, ok := e.Relational().OutputCols.Next(0) + if !ok { + panic(errors.AssertionFailedf("failed to retrieve column from %v", e.Op())) + } + if label, ok := jof.relLabels[firstCol]; ok { + return label + } + const lenAlphabet = 26 + labelCount := len(jof.relLabels) + label := string(rune(int('A') + (labelCount % lenAlphabet))) + number := labelCount / lenAlphabet + if number > 0 { + // Names will follow the pattern: A, B, ..., Z, A1, B1, etc. + label += strconv.Itoa(number) + } + jof.relLabels[firstCol] = label + return label +} + +// joinOpLabel returns an abbreviated string representation of a join operator. +func joinOpLabel(op opt.Operator) string { switch op { case opt.InnerJoinOp: return "inner" @@ -178,34 +229,3 @@ func outputOp(op opt.Operator) string { panic(errors.AssertionFailedf("unexpected operator: %v", op)) } } - -// outputRels returns a string with the aliases of the given base relations -// concatenated together. Panics if there is no alias for a base relation. -func outputRels(baseRels []memo.RelExpr, names map[opt.ColumnID]string) string { - buf := bytes.Buffer{} - for i := range baseRels { - firstCol, ok := baseRels[i].Relational().OutputCols.Next(0) - if !ok { - panic(errors.AssertionFailedf("failed to retrieve column from %v", baseRels[i].Op())) - } - buf.WriteString(names[firstCol]) - } - return buf.String() -} - -// getRelationName returns a simple alias for a base relation given the number -// of names generated so far. -func getRelationName(nameCount int) string { - const lenAlphabet = 26 - name := string(rune(int('A') + (nameCount % lenAlphabet))) - number := nameCount / lenAlphabet - if number > 0 { - // Names will follow the pattern: A, B, ..., Z, A1, B1, etc. - name += strconv.Itoa(number) - } - return name -} - -func separator(sep string) string { - return fmt.Sprintf("%s\n", strings.Repeat(sep, 80)) -} diff --git a/pkg/sql/opt/xform/join_order_builder.go b/pkg/sql/opt/xform/join_order_builder.go index 05b8df723fbf..e721f492fa3f 100644 --- a/pkg/sql/opt/xform/join_order_builder.go +++ b/pkg/sql/opt/xform/join_order_builder.go @@ -30,10 +30,32 @@ import ( type OnReorderFunc func( join memo.RelExpr, vertexes []memo.RelExpr, - edges []memo.FiltersExpr, - edgeOps []opt.Operator, + edges []OnReorderEdgeParam, ) +// OnReorderEdgeParam is a struct representing an edge in the join graph. This +// type is only used for the OnReorderFunc during testing and debugging. See the +// more efficient edge type that is used in the join reordering algorithm. +type OnReorderEdgeParam struct { + // Op is the original join operator from which the edge was constructed. + Op opt.Operator + // Filters is the edge's set of join filters + Filters memo.FiltersExpr + // SES is the edge's syntactic eligibility set. + SES []memo.RelExpr + // TES is the edge's total eligibility set. + TES []memo.RelExpr + // Rules is the set of conflict rules of the edge. + Rules []OnReorderRuleParam +} + +// OnReorderRuleParam is a struct representing a conflict rule. This type is +// only used for the OnReorderFunc during testing and debugging. See the more +// efficient conflictRule type that is used in the join reordering algorithm. +type OnReorderRuleParam struct { + From, To []memo.RelExpr +} + // OnAddJoinFunc defines the callback function for the NotifyOnAddJoin event // supported by JoinOrderBuilder. OnAddJoinFunc is called when JoinOrderBuilder // attempts to add a join to the memo via addJoin. The callback parameters give @@ -906,14 +928,23 @@ func (jb *JoinOrderBuilder) NotifyOnAddJoin(onAddJoin OnAddJoinFunc) { // function is nil. func (jb *JoinOrderBuilder) callOnReorderFunc(join memo.RelExpr) { // Get a slice with all edges of the join graph. - edgeSlice := make([]memo.FiltersExpr, 0, len(jb.edges)) - edgeOps := make([]opt.Operator, 0, len(jb.edges)) - for i := range jb.edges { - edgeSlice = append(edgeSlice, jb.edges[i].filters) - edgeOps = append(edgeOps, jb.edges[i].op.joinType) + edges := make([]OnReorderEdgeParam, 0, len(jb.edges)) + for _, edge := range jb.edges { + ep := OnReorderEdgeParam{ + Op: edge.op.joinType, + Filters: edge.filters, + SES: jb.getRelationSlice(edge.ses), + TES: jb.getRelationSlice(edge.tes), + } + for _, rule := range edge.rules { + ep.Rules = append(ep.Rules, OnReorderRuleParam{ + From: jb.getRelationSlice(rule.from), + To: jb.getRelationSlice(rule.to), + }) + } + edges = append(edges, ep) } - - jb.onReorderFunc(join, jb.getRelationSlice(jb.allVertexes()), edgeSlice, edgeOps) + jb.onReorderFunc(join, jb.getRelationSlice(jb.allVertexes()), edges) } // callOnAddJoinFunc calls the onAddJoinFunc callback function. Panics if the @@ -1328,24 +1359,6 @@ func commute(op opt.Operator) bool { // ON x = a // func assoc(edgeA, edgeB *edge) bool { - if edgeB.ses.intersects(edgeA.op.leftVertexes) || edgeA.ses.intersects(edgeB.op.rightVertexes) { - // Ensure that application of the associative property would not lead to - // 'orphaned' predicates, where one or more referenced relations are not in - // the resulting join's inputs. Take as an example this reordering that - // results from applying the associative property: - // - // SELECT * FROM (SELECT * FROM xy INNER JOIN ab ON y = a) - // INNER JOIN uv - // ON x = u - // => - // SELECT * FROM xy - // INNER JOIN (SELECT * FROM ab INNER JOIN uv ON x = u) - // ON y = a - // - // Note that the x = u predicate references the xy relation, which is not - // in that join's inputs. Therefore, this transformation is invalid. - return false - } return checkProperty(assocTable, edgeA, edgeB) } @@ -1368,11 +1381,6 @@ func assoc(edgeA, edgeB *edge) bool { // INNER JOIN ab ON x = a // func leftAsscom(edgeA, edgeB *edge) bool { - if edgeB.ses.intersects(edgeA.op.rightVertexes) || edgeA.ses.intersects(edgeB.op.rightVertexes) { - // Ensure that application of the left-asscom property would not lead to - // 'orphaned' predicates. See the assoc() comment for why this is necessary. - return false - } return checkProperty(leftAsscomTable, edgeA, edgeB) } @@ -1397,11 +1405,6 @@ func leftAsscom(edgeA, edgeB *edge) bool { // ON x = a // func rightAsscom(edgeA, edgeB *edge) bool { - if edgeB.ses.intersects(edgeA.op.leftVertexes) || edgeA.ses.intersects(edgeB.op.leftVertexes) { - // Ensure that application of the right-asscom property would not lead to - // 'orphaned' predicates. See the assoc() comment for why this is necessary. - return false - } return checkProperty(rightAsscomTable, edgeA, edgeB) } diff --git a/pkg/sql/opt/xform/join_order_builder_test.go b/pkg/sql/opt/xform/join_order_builder_test.go index f37e325f5870..a0786a59b77b 100644 --- a/pkg/sql/opt/xform/join_order_builder_test.go +++ b/pkg/sql/opt/xform/join_order_builder_test.go @@ -381,6 +381,22 @@ func TestJoinOrderBuilder_CalcTES(t *testing.T) { expectedTES: "ABC", expectedRules: "", }, + { // 20 + // SELECT * FROM ( + // SELECT * FROM ( + // SELECT * FROM A + // INNER JOIN B ON A.u = B.u + // ) INNER JOIN C ON B.v = C.v + // ) INNER JOIN D ON A.w = D.w + rootEdge: testEdge{joinOp: opt.InnerJoinOp, left: "ABC", right: "D", ses: "AD", notNull: "AD"}, + leftChildEdges: []testEdge{ + {joinOp: opt.InnerJoinOp, left: "AB", right: "C", ses: "BC", notNull: "BC"}, + {joinOp: opt.InnerJoinOp, left: "A", right: "B", ses: "AB", notNull: "AB"}, + }, + rightChildEdges: []testEdge{}, + expectedTES: "AD", + expectedRules: "", + }, } for i, tc := range testCases { diff --git a/pkg/sql/opt/xform/testdata/external/tpce b/pkg/sql/opt/xform/testdata/external/tpce index ae2c7a5cb80c..38650c3b3a14 100644 --- a/pkg/sql/opt/xform/testdata/external/tpce +++ b/pkg/sql/opt/xform/testdata/external/tpce @@ -832,7 +832,7 @@ project │ │ │ ├── fd: ()-->(10), (11)-->(12), (18)-->(20), (25)-->(31), (18)==(9,11,25), (25)==(9,11,18), (11)==(9,18,25), (9)==(11,18,25) │ │ │ ├── inner-join (lookup security) │ │ │ │ ├── columns: symb:9!null dm_date:10!null dm_s_symb:11!null dm_close:12!null s_symb:25!null s_num_out:31!null - │ │ │ │ ├── key columns: [11] = [25] + │ │ │ │ ├── key columns: [9] = [25] │ │ │ │ ├── lookup columns are key │ │ │ │ ├── fd: ()-->(10), (11)-->(12), (25)-->(31), (11)==(9,25), (25)==(9,11), (9)==(11,25) │ │ │ │ ├── inner-join (lookup daily_market) @@ -1004,7 +1004,7 @@ project │ │ │ ├── fd: ()-->(36), (37)-->(38), (44)-->(46), (51)-->(57), (44)==(35,37,51), (51)==(35,37,44), (37)==(35,44,51), (35)==(37,44,51) │ │ │ ├── inner-join (lookup security) │ │ │ │ ├── columns: symb:35!null dm_date:36!null dm_s_symb:37!null dm_close:38!null s_symb:51!null s_num_out:57!null - │ │ │ │ ├── key columns: [37] = [51] + │ │ │ │ ├── key columns: [35] = [51] │ │ │ │ ├── lookup columns are key │ │ │ │ ├── key: (51) │ │ │ │ ├── fd: ()-->(36), (37)-->(38), (51)-->(57), (37)==(35,51), (51)==(35,37), (35)==(37,51) @@ -1198,7 +1198,7 @@ project │ │ │ ├── fd: ()-->(7), (8)-->(9), (15)-->(17), (22)-->(28), (15)==(6,8,22), (22)==(6,8,15), (8)==(6,15,22), (6)==(8,15,22) │ │ │ ├── inner-join (lookup security) │ │ │ │ ├── columns: symb:6!null dm_date:7!null dm_s_symb:8!null dm_close:9!null s_symb:22!null s_num_out:28!null - │ │ │ │ ├── key columns: [8] = [22] + │ │ │ │ ├── key columns: [6] = [22] │ │ │ │ ├── lookup columns are key │ │ │ │ ├── key: (22) │ │ │ │ ├── fd: ()-->(7), (8)-->(9), (22)-->(28), (8)==(6,22), (22)==(6,8), (6)==(8,22) diff --git a/pkg/sql/opt/xform/testdata/rules/join_order b/pkg/sql/opt/xform/testdata/rules/join_order index 2e0e4e52118e..a02294127b6f 100644 --- a/pkg/sql/opt/xform/testdata/rules/join_order +++ b/pkg/sql/opt/xform/testdata/rules/join_order @@ -657,7 +657,7 @@ memo (optimized, ~62KB, required=[presentation: b:1,x:2,c:5,y:6,d:9,z:10,a:13,b: │ └── [] │ ├── best: (scan dz,cols=(9,10)) │ └── cost: 1064.42 - ├── G11: (inner-join G2 G14 G4) (inner-join G14 G2 G4) (inner-join G5 G16 G4) (inner-join G16 G5 G4) (inner-join G8 G18 G23) (inner-join G18 G8 G23) (merge-join G14 G2 G19 inner-join,+6,+2) (merge-join G16 G5 G19 inner-join,+2,+6) (lookup-join G8 G19 abc,keyCols=[6],outCols=(1,2,5,6,13-16)) (merge-join G18 G8 G19 inner-join,+13,+6) + ├── G11: (inner-join G2 G14 G4) (inner-join G14 G2 G4) (inner-join G5 G16 G4) (inner-join G16 G5 G4) (inner-join G8 G18 G23) (inner-join G18 G8 G23) (merge-join G14 G2 G19 inner-join,+6,+2) (merge-join G16 G5 G19 inner-join,+2,+6) (lookup-join G8 G19 abc,keyCols=[2],outCols=(1,2,5,6,13-16)) (merge-join G18 G8 G19 inner-join,+13,+2) │ ├── [ordering: +(2|6|13)] │ │ ├── best: (merge-join G14="[ordering: +(6|13)]" G2="[ordering: +2]" G19 inner-join,+6,+2) │ │ └── cost: 3860.48 @@ -672,12 +672,12 @@ memo (optimized, ~62KB, required=[presentation: b:1,x:2,c:5,y:6,d:9,z:10,a:13,b: │ └── [] │ ├── best: (inner-join G2 G10 G21) │ └── cost: 2257.02 - ├── G14: (inner-join G5 G18 G23) (inner-join G18 G5 G23) (lookup-join G5 G19 abc,keyCols=[6],outCols=(5,6,13-16)) (merge-join G18 G5 G19 inner-join,+13,+6) + ├── G14: (inner-join G5 G18 G25) (inner-join G18 G5 G25) (lookup-join G5 G19 abc,keyCols=[6],outCols=(5,6,13-16)) (merge-join G18 G5 G19 inner-join,+13,+6) │ ├── [ordering: +(6|13)] │ │ ├── best: (merge-join G18="[ordering: +13]" G5="[ordering: +6]" G19 inner-join,+13,+6) │ │ └── cost: 2438.64 │ └── [] - │ ├── best: (inner-join G5 G18 G23) + │ ├── best: (inner-join G5 G18 G25) │ └── cost: 2209.31 ├── G15: (inner-join G5 G10 G7) (inner-join G10 G5 G7) │ ├── [ordering: +(6|10)] @@ -686,12 +686,12 @@ memo (optimized, ~62KB, required=[presentation: b:1,x:2,c:5,y:6,d:9,z:10,a:13,b: │ └── [] │ ├── best: (inner-join G5 G10 G7) │ └── cost: 2257.02 - ├── G16: (inner-join G2 G18 G25) (inner-join G18 G2 G25) (lookup-join G2 G19 abc,keyCols=[2],outCols=(1,2,13-16)) (merge-join G18 G2 G19 inner-join,+13,+2) + ├── G16: (inner-join G2 G18 G23) (inner-join G18 G2 G23) (lookup-join G2 G19 abc,keyCols=[2],outCols=(1,2,13-16)) (merge-join G18 G2 G19 inner-join,+13,+2) │ ├── [ordering: +(2|13)] │ │ ├── best: (merge-join G18="[ordering: +13]" G2="[ordering: +2]" G19 inner-join,+13,+2) │ │ └── cost: 2438.64 │ └── [] - │ ├── best: (inner-join G2 G18 G25) + │ ├── best: (inner-join G2 G18 G23) │ └── cost: 2209.31 ├── G17: (inner-join G2 G15 G4) (inner-join G15 G2 G4) (inner-join G5 G13 G7) (inner-join G13 G5 G7) (inner-join G8 G10 G7) (inner-join G10 G8 G7) │ ├── [ordering: +(2|6|10)] @@ -718,9 +718,9 @@ memo (optimized, ~62KB, required=[presentation: b:1,x:2,c:5,y:6,d:9,z:10,a:13,b: ├── G27: (variable y) ├── G28: (eq G26 G29) ├── G29: (variable z) - ├── G30: (eq G27 G31) + ├── G30: (eq G26 G31) ├── G31: (variable a) - └── G32: (eq G26 G31) + └── G32: (eq G27 G31) opt SELECT * FROM bx, cy, dz, abc WHERE x = y AND y = z AND z = a @@ -848,83 +848,74 @@ SELECT * FROM bx INNER JOIN cy ON b = c INNER JOIN (SELECT * FROM dz WHERE z > 0) ON c = d ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -inner-join (hash) - ├── scan bx - ├── scan cy - └── filters - └── b = c - -----Vertexes---- -A: -scan bx - -B: -scan cy - -----Edges---- -b = c [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] - +Join Tree #1 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── scan bx + ├── scan cy + └── filters + └── b = c +Vertexes + A: + scan bx + B: + scan cy +Edges + b = c [inner, ses=AB, tes=AB, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #2---- -inner-join (hash) - ├── inner-join (hash) - │ ├── scan bx - │ ├── scan cy - │ └── filters - │ └── b = c - ├── select - │ ├── scan dz - │ └── filters - │ └── z > 0 - └── filters - └── c = d - -----Vertexes---- -A: -scan bx - -B: -scan cy - -C: -select - ├── scan dz - └── filters - └── z > 0 - -----Edges---- -b = c [inner] -c = d [inner] -b = d [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] -----Joining AC---- -A C refs [AC] [inner] -C A refs [AC] [inner] -----Joining BC---- -B C refs [BC] [inner] -C B refs [BC] [inner] -----Joining ABC---- -A BC refs [AB] [inner] -BC A refs [AB] [inner] -B AC refs [AB] [inner] -AC B refs [AB] [inner] -AB C refs [BC] [inner] -C AB refs [BC] [inner] - -Joins Considered: 12 +Join Tree #2 -------------------------------------------------------------------------------- -----Final Plan---- + inner-join (hash) + ├── inner-join (hash) + │ ├── scan bx + │ ├── scan cy + │ └── filters + │ └── b = c + ├── select + │ ├── scan dz + │ └── filters + │ └── z > 0 + └── filters + └── c = d +Vertexes + A: + scan bx + B: + scan cy + C: + select + ├── scan dz + └── filters + └── z > 0 +Edges + b = c [inner, ses=AB, tes=AB, rules=()] + c = d [inner, ses=BC, tes=BC, rules=()] + b = d [inner, ses=AC, tes=AC, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining AC + A C [inner, refs=AC] + C A [inner, refs=AC] +Joining BC + B C [inner, refs=BC] + C B [inner, refs=BC] +Joining ABC + A BC [inner, refs=AB] + BC A [inner, refs=AB] + B AC [inner, refs=AB] + AC B [inner, refs=AB] + AB C [inner, refs=BC] + C AB [inner, refs=BC] +Joins Considered: 12 +================================================================================ +Final Plan +================================================================================ inner-join (merge) ├── scan bx ├── inner-join (merge) @@ -935,9 +926,6 @@ inner-join (merge) │ │ └── z > 0 │ └── filters (true) └── filters (true) --------------------------------------------------------------------------------- ----- ----- reorderjoins format=hide-all SELECT * FROM bx @@ -945,162 +933,147 @@ INNER JOIN cy ON b = c INNER JOIN (SELECT max(z) AS m FROM dz) ON y = m INNER JOIN abc ON m = a ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -inner-join (hash) - ├── scan bx - ├── scan cy - └── filters - └── bx.b = cy.c - -----Vertexes---- -A: -scan bx - -B: -scan cy - -----Edges---- -bx.b = cy.c [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] - +Join Tree #1 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── scan bx + ├── scan cy + └── filters + └── bx.b = cy.c +Vertexes + A: + scan bx + B: + scan cy +Edges + bx.b = cy.c [inner, ses=AB, tes=AB, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #2---- -inner-join (hash) - ├── inner-join (hash) - │ ├── scan bx - │ ├── scan cy - │ └── filters - │ └── bx.b = cy.c - ├── scalar-group-by - │ ├── scan dz - │ └── aggregations - │ └── max - │ └── z - └── filters - └── y = max - -----Vertexes---- -A: -scan bx - -B: -scan cy - -C: -scalar-group-by - ├── scan dz - └── aggregations - └── max - └── z - -----Edges---- -bx.b = cy.c [inner] -y = max [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] -----Joining BC---- -B C refs [BC] [inner] -C B refs [BC] [inner] -----Joining ABC---- -A BC refs [AB] [inner] -BC A refs [AB] [inner] -AB C refs [BC] [inner] -C AB refs [BC] [inner] - +Join Tree #2 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── inner-join (hash) + │ ├── scan bx + │ ├── scan cy + │ └── filters + │ └── bx.b = cy.c + ├── scalar-group-by + │ ├── scan dz + │ └── aggregations + │ └── max + │ └── z + └── filters + └── y = max +Vertexes + A: + scan bx + B: + scan cy + C: + scalar-group-by + ├── scan dz + └── aggregations + └── max + └── z +Edges + bx.b = cy.c [inner, ses=AB, tes=AB, rules=()] + y = max [inner, ses=BC, tes=BC, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining BC + B C [inner, refs=BC] + C B [inner, refs=BC] +Joining ABC + A BC [inner, refs=AB] + BC A [inner, refs=AB] + AB C [inner, refs=BC] + C AB [inner, refs=BC] Joins Considered: 8 -------------------------------------------------------------------------------- -----Join Tree #3---- -inner-join (hash) - ├── inner-join (hash) - │ ├── inner-join (hash) - │ │ ├── scan bx - │ │ ├── scan cy - │ │ └── filters - │ │ └── bx.b = cy.c - │ ├── scalar-group-by - │ │ ├── scan dz - │ │ └── aggregations - │ │ └── max - │ │ └── z - │ └── filters - │ └── y = max - ├── scan abc - └── filters - └── max = a - -----Vertexes---- -A: -scan bx - -B: -scan cy - -C: -scalar-group-by - ├── scan dz - └── aggregations - └── max - └── z - -D: -scan abc - -----Edges---- -bx.b = cy.c [inner] -y = max [inner] -max = a [inner] -y = a [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] -----Joining BC---- -B C refs [BC] [inner] -C B refs [BC] [inner] -----Joining ABC---- -A BC refs [AB] [inner] -BC A refs [AB] [inner] -AB C refs [BC] [inner] -C AB refs [BC] [inner] -----Joining BD---- -B D refs [BD] [inner] -D B refs [BD] [inner] -----Joining ABD---- -A BD refs [AB] [inner] -BD A refs [AB] [inner] -AB D refs [BD] [inner] -D AB refs [BD] [inner] -----Joining CD---- -C D refs [CD] [inner] -D C refs [CD] [inner] -----Joining BCD---- -B CD refs [BC] [inner] -CD B refs [BC] [inner] -C BD refs [BC] [inner] -BD C refs [BC] [inner] -BC D refs [CD] [inner] -D BC refs [CD] [inner] -----Joining ABCD---- -A BCD refs [AB] [inner] -BCD A refs [AB] [inner] -AB CD refs [BC] [inner] -CD AB refs [BC] [inner] -C ABD refs [BC] [inner] -ABD C refs [BC] [inner] -ABC D refs [CD] [inner] -D ABC refs [CD] [inner] - -Joins Considered: 30 +Join Tree #3 -------------------------------------------------------------------------------- -----Final Plan---- + inner-join (hash) + ├── inner-join (hash) + │ ├── inner-join (hash) + │ │ ├── scan bx + │ │ ├── scan cy + │ │ └── filters + │ │ └── bx.b = cy.c + │ ├── scalar-group-by + │ │ ├── scan dz + │ │ └── aggregations + │ │ └── max + │ │ └── z + │ └── filters + │ └── y = max + ├── scan abc + └── filters + └── max = a +Vertexes + A: + scan bx + B: + scan cy + C: + scalar-group-by + ├── scan dz + └── aggregations + └── max + └── z + D: + scan abc +Edges + bx.b = cy.c [inner, ses=AB, tes=AB, rules=()] + y = max [inner, ses=BC, tes=BC, rules=()] + max = a [inner, ses=CD, tes=CD, rules=()] + y = a [inner, ses=BD, tes=BD, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining BC + B C [inner, refs=BC] + C B [inner, refs=BC] +Joining ABC + A BC [inner, refs=AB] + BC A [inner, refs=AB] + AB C [inner, refs=BC] + C AB [inner, refs=BC] +Joining BD + B D [inner, refs=BD] + D B [inner, refs=BD] +Joining ABD + A BD [inner, refs=AB] + BD A [inner, refs=AB] + AB D [inner, refs=BD] + D AB [inner, refs=BD] +Joining CD + C D [inner, refs=CD] + D C [inner, refs=CD] +Joining BCD + B CD [inner, refs=BC] + CD B [inner, refs=BC] + C BD [inner, refs=BC] + BD C [inner, refs=BC] + BC D [inner, refs=CD] + D BC [inner, refs=CD] +Joining ABCD + A BCD [inner, refs=AB] + BCD A [inner, refs=AB] + AB CD [inner, refs=BC] + CD AB [inner, refs=BC] + C ABD [inner, refs=BC] + ABD C [inner, refs=BC] + ABC D [inner, refs=CD] + D ABC [inner, refs=CD] +Joins Considered: 30 +================================================================================ +Final Plan +================================================================================ inner-join (lookup bx) ├── lookup columns are key ├── inner-join (hash) @@ -1116,9 +1089,6 @@ inner-join (lookup bx) │ └── filters │ └── y = max └── filters (true) --------------------------------------------------------------------------------- ----- ----- # Treat the join with hints as a base relation. Note that the implicit edges # x = a and y = a are added to the join graph. The x = y filter is not an edge @@ -1130,100 +1100,91 @@ INNER HASH JOIN cy ON x = y INNER JOIN dz ON y = z INNER JOIN abc ON z = a ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -inner-join (hash) - ├── inner-join (hash) - │ ├── flags: force hash join (store right side) - │ ├── scan bx - │ ├── scan cy - │ └── filters - │ └── x = y - ├── scan dz - └── filters - └── y = z - -----Vertexes---- -A: -inner-join (hash) - ├── flags: force hash join (store right side) - ├── scan bx - ├── scan cy - └── filters - └── x = y - -B: -scan dz - -----Edges---- -y = z [inner] -x = z [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] - +Join Tree #1 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── inner-join (hash) + │ ├── flags: force hash join (store right side) + │ ├── scan bx + │ ├── scan cy + │ └── filters + │ └── x = y + ├── scan dz + └── filters + └── y = z +Vertexes + A: + inner-join (hash) + ├── flags: force hash join (store right side) + ├── scan bx + ├── scan cy + └── filters + └── x = y + B: + scan dz +Edges + y = z [inner, ses=AB, tes=AB, rules=()] + x = z [inner, ses=AB, tes=AB, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #2---- -inner-join (hash) - ├── inner-join (hash) - │ ├── inner-join (hash) - │ │ ├── flags: force hash join (store right side) - │ │ ├── scan bx - │ │ ├── scan cy - │ │ └── filters - │ │ └── x = y - │ ├── scan dz - │ └── filters - │ └── y = z - ├── scan abc - └── filters - └── z = a - -----Vertexes---- -A: -inner-join (hash) - ├── flags: force hash join (store right side) - ├── scan bx - ├── scan cy - └── filters - └── x = y - -B: -scan dz - -C: -scan abc - -----Edges---- -y = z [inner] -z = a [inner] -x = z [inner] -x = a [inner] -y = a [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] -----Joining AC---- -A C refs [AC] [inner] -C A refs [AC] [inner] -----Joining BC---- -B C refs [BC] [inner] -C B refs [BC] [inner] -----Joining ABC---- -A BC refs [AB] [inner] -BC A refs [AB] [inner] -B AC refs [AB] [inner] -AC B refs [AB] [inner] -AB C refs [BC] [inner] -C AB refs [BC] [inner] - -Joins Considered: 12 +Join Tree #2 -------------------------------------------------------------------------------- -----Final Plan---- + inner-join (hash) + ├── inner-join (hash) + │ ├── inner-join (hash) + │ │ ├── flags: force hash join (store right side) + │ │ ├── scan bx + │ │ ├── scan cy + │ │ └── filters + │ │ └── x = y + │ ├── scan dz + │ └── filters + │ └── y = z + ├── scan abc + └── filters + └── z = a +Vertexes + A: + inner-join (hash) + ├── flags: force hash join (store right side) + ├── scan bx + ├── scan cy + └── filters + └── x = y + B: + scan dz + C: + scan abc +Edges + y = z [inner, ses=AB, tes=AB, rules=()] + z = a [inner, ses=BC, tes=BC, rules=()] + x = z [inner, ses=AB, tes=AB, rules=()] + x = a [inner, ses=AC, tes=AC, rules=()] + y = a [inner, ses=AC, tes=AC, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining AC + A C [inner, refs=AC] + C A [inner, refs=AC] +Joining BC + B C [inner, refs=BC] + C B [inner, refs=BC] +Joining ABC + A BC [inner, refs=AB] + BC A [inner, refs=AB] + B AC [inner, refs=AB] + AC B [inner, refs=AB] + AB C [inner, refs=BC] + C AB [inner, refs=BC] +Joins Considered: 12 +================================================================================ +Final Plan +================================================================================ inner-join (hash) ├── inner-join (hash) │ ├── flags: force hash join (store right side) @@ -1238,9 +1199,6 @@ inner-join (hash) │ └── z = a └── filters └── y = z --------------------------------------------------------------------------------- ----- ----- # Ignore the apply join. However, all other joins can be reordered despite the # presence of outer columns. @@ -1255,134 +1213,119 @@ INNER JOIN LATERAL ) ON x = y ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -inner-join (hash) - ├── scan cy - ├── scan dz - └── filters - └── y = z - -----Vertexes---- -A: -scan cy - -B: -scan dz - -----Edges---- -y = z [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] - +Join Tree #1 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── scan cy + ├── scan dz + └── filters + └── y = z +Vertexes + A: + scan cy + B: + scan dz +Edges + y = z [inner, ses=AB, tes=AB, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #2---- -inner-join (hash) - ├── inner-join (hash) - │ ├── scan cy - │ ├── scan dz - │ └── filters - │ └── y = z - ├── scan abc - └── filters - └── z = a - -----Vertexes---- -A: -scan cy - -B: -scan dz - -C: -scan abc - -----Edges---- -y = z [inner] -z = a [inner] -y = a [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] -----Joining AC---- -A C refs [AC] [inner] -C A refs [AC] [inner] -----Joining BC---- -B C refs [BC] [inner] -C B refs [BC] [inner] -----Joining ABC---- -A BC refs [AB] [inner] -BC A refs [AB] [inner] -B AC refs [AB] [inner] -AC B refs [AB] [inner] -AB C refs [BC] [inner] -C AB refs [BC] [inner] - +Join Tree #2 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── inner-join (hash) + │ ├── scan cy + │ ├── scan dz + │ └── filters + │ └── y = z + ├── scan abc + └── filters + └── z = a +Vertexes + A: + scan cy + B: + scan dz + C: + scan abc +Edges + y = z [inner, ses=AB, tes=AB, rules=()] + z = a [inner, ses=BC, tes=BC, rules=()] + y = a [inner, ses=AC, tes=AC, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining AC + A C [inner, refs=AC] + C A [inner, refs=AC] +Joining BC + B C [inner, refs=BC] + C B [inner, refs=BC] +Joining ABC + A BC [inner, refs=AB] + BC A [inner, refs=AB] + B AC [inner, refs=AB] + AC B [inner, refs=AB] + AB C [inner, refs=BC] + C AB [inner, refs=BC] Joins Considered: 12 -------------------------------------------------------------------------------- -----Join Tree #3---- -inner-join (cross) - ├── values - │ └── (x,) - ├── inner-join (hash) - │ ├── inner-join (hash) - │ │ ├── scan cy - │ │ ├── scan dz - │ │ └── filters - │ │ └── y = z - │ ├── scan abc - │ └── filters - │ └── z = a - └── filters (true) - -----Vertexes---- -D: -values - └── (x,) - -A: -scan cy - -B: -scan dz - -C: -scan abc - -----Edges---- -y = z [inner] -z = a [inner] -cross [inner] -y = a [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] -----Joining AC---- -A C refs [AC] [inner] -C A refs [AC] [inner] -----Joining BC---- -B C refs [BC] [inner] -C B refs [BC] [inner] -----Joining ABC---- -A BC refs [AB] [inner] -BC A refs [AB] [inner] -B AC refs [AB] [inner] -AC B refs [AB] [inner] -AB C refs [BC] [inner] -C AB refs [BC] [inner] -----Joining DABC---- -D ABC refs [] [inner] -ABC D refs [] [inner] - -Joins Considered: 14 +Join Tree #3 -------------------------------------------------------------------------------- -----Final Plan---- + inner-join (cross) + ├── values + │ └── (x,) + ├── inner-join (hash) + │ ├── inner-join (hash) + │ │ ├── scan cy + │ │ ├── scan dz + │ │ └── filters + │ │ └── y = z + │ ├── scan abc + │ └── filters + │ └── z = a + └── filters (true) +Vertexes + D: + values + └── (x,) + A: + scan cy + B: + scan dz + C: + scan abc +Edges + y = z [inner, ses=AB, tes=AB, rules=()] + z = a [inner, ses=BC, tes=BC, rules=()] + cross [inner, ses=, tes=DABC, rules=()] + y = a [inner, ses=AC, tes=AC, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining AC + A C [inner, refs=AC] + C A [inner, refs=AC] +Joining BC + B C [inner, refs=BC] + C B [inner, refs=BC] +Joining ABC + A BC [inner, refs=AB] + BC A [inner, refs=AB] + B AC [inner, refs=AB] + AC B [inner, refs=AB] + AB C [inner, refs=BC] + C AB [inner, refs=BC] +Joining DABC + D ABC [inner, refs=] + ABC D [inner, refs=] +Joins Considered: 14 +================================================================================ +Final Plan +================================================================================ inner-join-apply ├── scan bx ├── inner-join (cross) @@ -1400,9 +1343,6 @@ inner-join-apply │ └── filters (true) └── filters └── x = y --------------------------------------------------------------------------------- ----- ----- reorderjoins format=hide-all SELECT * FROM @@ -1419,131 +1359,117 @@ INNER JOIN ) ON abc_b = bx_b AND abc_c = cy_c ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -inner-join (hash) - ├── scan bx - ├── scan cy - └── filters - └── x = y - -----Vertexes---- -A: -scan bx - -B: -scan cy - -----Edges---- -x = y [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] - +Join Tree #1 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── scan bx + ├── scan cy + └── filters + └── x = y +Vertexes + A: + scan bx + B: + scan cy +Edges + x = y [inner, ses=AB, tes=AB, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #2---- -inner-join (hash) - ├── scan abc - ├── scan dz - └── filters - └── a = z - -----Vertexes---- -C: -scan abc - -D: -scan dz - -----Edges---- -a = z [inner] - -----Joining CD---- -C D refs [CD] [inner] -D C refs [CD] [inner] - +Join Tree #2 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── scan abc + ├── scan dz + └── filters + └── a = z +Vertexes + C: + scan abc + D: + scan dz +Edges + a = z [inner, ses=CD, tes=CD, rules=()] +Joining CD + C D [inner, refs=CD] + D C [inner, refs=CD] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #3---- -inner-join (hash) - ├── inner-join (hash) - │ ├── scan bx - │ ├── scan cy - │ └── filters - │ └── x = y - ├── inner-join (hash) - │ ├── scan abc - │ ├── scan dz - │ └── filters - │ └── a = z - └── filters - ├── abc.b = bx.b - └── abc.c = cy.c - -----Vertexes---- -A: -scan bx - -B: -scan cy - -C: -scan abc - -D: -scan dz - -----Edges---- -x = y [inner] -a = z [inner] -abc.b = bx.b [inner] -abc.c = cy.c [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] -----Joining AC---- -A C refs [AC] [inner] -C A refs [AC] [inner] -----Joining BC---- -B C refs [BC] [inner] -C B refs [BC] [inner] -----Joining ABC---- -A BC refs [ABC] [inner] -BC A refs [ABC] [inner] -B AC refs [ABC] [inner] -AC B refs [ABC] [inner] -AB C refs [ABC] [inner] -C AB refs [ABC] [inner] -----Joining CD---- -C D refs [CD] [inner] -D C refs [CD] [inner] -----Joining ACD---- -A CD refs [AC] [inner] -CD A refs [AC] [inner] -AC D refs [CD] [inner] -D AC refs [CD] [inner] -----Joining BCD---- -B CD refs [BC] [inner] -CD B refs [BC] [inner] -BC D refs [CD] [inner] -D BC refs [CD] [inner] -----Joining ABCD---- -A BCD refs [ABC] [inner] -BCD A refs [ABC] [inner] -B ACD refs [ABC] [inner] -ACD B refs [ABC] [inner] -AB CD refs [ABC] [inner] -CD AB refs [ABC] [inner] -ABC D refs [CD] [inner] -D ABC refs [CD] [inner] - -Joins Considered: 30 +Join Tree #3 -------------------------------------------------------------------------------- -----Final Plan---- + inner-join (hash) + ├── inner-join (hash) + │ ├── scan bx + │ ├── scan cy + │ └── filters + │ └── x = y + ├── inner-join (hash) + │ ├── scan abc + │ ├── scan dz + │ └── filters + │ └── a = z + └── filters + ├── abc.b = bx.b + └── abc.c = cy.c +Vertexes + A: + scan bx + B: + scan cy + C: + scan abc + D: + scan dz +Edges + x = y [inner, ses=AB, tes=AB, rules=()] + a = z [inner, ses=CD, tes=CD, rules=()] + abc.b = bx.b [inner, ses=AC, tes=AC, rules=()] + abc.c = cy.c [inner, ses=BC, tes=BC, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining AC + A C [inner, refs=AC] + C A [inner, refs=AC] +Joining BC + B C [inner, refs=BC] + C B [inner, refs=BC] +Joining ABC + A BC [inner, refs=ABC] + BC A [inner, refs=ABC] + B AC [inner, refs=ABC] + AC B [inner, refs=ABC] + AB C [inner, refs=ABC] + C AB [inner, refs=ABC] +Joining CD + C D [inner, refs=CD] + D C [inner, refs=CD] +Joining ACD + A CD [inner, refs=AC] + CD A [inner, refs=AC] + AC D [inner, refs=CD] + D AC [inner, refs=CD] +Joining BCD + B CD [inner, refs=BC] + CD B [inner, refs=BC] + BC D [inner, refs=CD] + D BC [inner, refs=CD] +Joining ABCD + A BCD [inner, refs=ABC] + BCD A [inner, refs=ABC] + B ACD [inner, refs=ABC] + ACD B [inner, refs=ABC] + AB CD [inner, refs=ABC] + CD AB [inner, refs=ABC] + ABC D [inner, refs=CD] + D ABC [inner, refs=CD] +Joins Considered: 30 +================================================================================ +Final Plan +================================================================================ project └── inner-join (hash) ├── scan dz @@ -1559,9 +1485,6 @@ project │ └── abc.b = bx.b └── filters └── a = z --------------------------------------------------------------------------------- ----- ----- reorderjoins format=hide-all SELECT * @@ -1569,69 +1492,60 @@ FROM bx INNER JOIN cy ON b = c LEFT JOIN dz ON x = z ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -inner-join (hash) - ├── scan bx - ├── scan cy - └── filters - └── b = c - -----Vertexes---- -A: -scan bx - -B: -scan cy - -----Edges---- -b = c [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] - +Join Tree #1 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── scan bx + ├── scan cy + └── filters + └── b = c +Vertexes + A: + scan bx + B: + scan cy +Edges + b = c [inner, ses=AB, tes=AB, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #2---- -left-join (hash) - ├── inner-join (hash) - │ ├── scan bx - │ ├── scan cy - │ └── filters - │ └── b = c - ├── scan dz - └── filters - └── x = z - -----Vertexes---- -A: -scan bx - -B: -scan cy - -C: -scan dz - -----Edges---- -b = c [inner] -x = z [left] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] -----Joining AC---- -A C refs [AC] [left] -----Joining ABC---- -B AC refs [AB] [inner] -AC B refs [AB] [inner] -AB C refs [AC] [left] - -Joins Considered: 6 +Join Tree #2 -------------------------------------------------------------------------------- -----Final Plan---- + left-join (hash) + ├── inner-join (hash) + │ ├── scan bx + │ ├── scan cy + │ └── filters + │ └── b = c + ├── scan dz + └── filters + └── x = z +Vertexes + A: + scan bx + B: + scan cy + C: + scan dz +Edges + b = c [inner, ses=AB, tes=AB, rules=()] + x = z [left, ses=AC, tes=AC, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining AC + A C [left, refs=AC] +Joining ABC + B AC [inner, refs=AB] + AC B [inner, refs=AB] + AB C [left, refs=AC] +Joins Considered: 6 +================================================================================ +Final Plan +================================================================================ left-join (hash) ├── inner-join (merge) │ ├── scan bx @@ -1640,9 +1554,6 @@ left-join (hash) ├── scan dz └── filters └── x = z --------------------------------------------------------------------------------- ----- ----- # Left-asscom property for left joins. reorderjoins format=hide-all @@ -1651,66 +1562,57 @@ FROM bx LEFT JOIN cy ON b = c LEFT JOIN dz ON x = z ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -left-join (hash) - ├── scan bx - ├── scan cy - └── filters - └── b = c - -----Vertexes---- -A: -scan bx - -B: -scan cy - -----Edges---- -b = c [left] - -----Joining AB---- -A B refs [AB] [left] - +Join Tree #1 +-------------------------------------------------------------------------------- + left-join (hash) + ├── scan bx + ├── scan cy + └── filters + └── b = c +Vertexes + A: + scan bx + B: + scan cy +Edges + b = c [left, ses=AB, tes=AB, rules=()] +Joining AB + A B [left, refs=AB] Joins Considered: 1 -------------------------------------------------------------------------------- -----Join Tree #2---- -left-join (hash) - ├── left-join (hash) - │ ├── scan bx - │ ├── scan cy - │ └── filters - │ └── b = c - ├── scan dz - └── filters - └── x = z - -----Vertexes---- -A: -scan bx - -B: -scan cy - -C: -scan dz - -----Edges---- -b = c [left] -x = z [left] - -----Joining AB---- -A B refs [AB] [left] -----Joining AC---- -A C refs [AC] [left] -----Joining ABC---- -AC B refs [AB] [left] -AB C refs [AC] [left] - -Joins Considered: 4 +Join Tree #2 -------------------------------------------------------------------------------- -----Final Plan---- + left-join (hash) + ├── left-join (hash) + │ ├── scan bx + │ ├── scan cy + │ └── filters + │ └── b = c + ├── scan dz + └── filters + └── x = z +Vertexes + A: + scan bx + B: + scan cy + C: + scan dz +Edges + b = c [left, ses=AB, tes=AB, rules=()] + x = z [left, ses=AC, tes=AC, rules=()] +Joining AB + A B [left, refs=AB] +Joining AC + A C [left, refs=AC] +Joining ABC + AC B [left, refs=AB] + AB C [left, refs=AC] +Joins Considered: 4 +================================================================================ +Final Plan +================================================================================ left-join (hash) ├── left-join (merge) │ ├── scan bx @@ -1719,9 +1621,6 @@ left-join (hash) ├── scan dz └── filters └── x = z --------------------------------------------------------------------------------- ----- ----- # Left-asscom property does not apply when the upper left join references the # right side of the lower. However, associative property does apply when the @@ -1733,66 +1632,57 @@ FROM bx LEFT JOIN cy ON b = c LEFT JOIN dz ON y = z ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -left-join (hash) - ├── scan bx - ├── scan cy - └── filters - └── b = c - -----Vertexes---- -A: -scan bx - -B: -scan cy - -----Edges---- -b = c [left] - -----Joining AB---- -A B refs [AB] [left] - +Join Tree #1 +-------------------------------------------------------------------------------- + left-join (hash) + ├── scan bx + ├── scan cy + └── filters + └── b = c +Vertexes + A: + scan bx + B: + scan cy +Edges + b = c [left, ses=AB, tes=AB, rules=()] +Joining AB + A B [left, refs=AB] Joins Considered: 1 -------------------------------------------------------------------------------- -----Join Tree #2---- -left-join (hash) - ├── left-join (hash) - │ ├── scan bx - │ ├── scan cy - │ └── filters - │ └── b = c - ├── scan dz - └── filters - └── y = z - -----Vertexes---- -A: -scan bx - -B: -scan cy - -C: -scan dz - -----Edges---- -b = c [left] -y = z [left] - -----Joining AB---- -A B refs [AB] [left] -----Joining BC---- -B C refs [BC] [left] -----Joining ABC---- -A BC refs [AB] [left] -AB C refs [BC] [left] - -Joins Considered: 4 +Join Tree #2 -------------------------------------------------------------------------------- -----Final Plan---- + left-join (hash) + ├── left-join (hash) + │ ├── scan bx + │ ├── scan cy + │ └── filters + │ └── b = c + ├── scan dz + └── filters + └── y = z +Vertexes + A: + scan bx + B: + scan cy + C: + scan dz +Edges + b = c [left, ses=AB, tes=AB, rules=()] + y = z [left, ses=BC, tes=BC, rules=()] +Joining AB + A B [left, refs=AB] +Joining BC + B C [left, refs=BC] +Joining ABC + A BC [left, refs=AB] + AB C [left, refs=BC] +Joins Considered: 4 +================================================================================ +Final Plan +================================================================================ left-join (hash) ├── left-join (merge) │ ├── scan bx @@ -1801,9 +1691,6 @@ left-join (hash) ├── scan dz └── filters └── y = z --------------------------------------------------------------------------------- ----- ----- # Left, semi and anti join tree. Multiple applications of left-asscom are # possible. Note that the inner join results from commuting the semi join. @@ -1815,158 +1702,138 @@ WHERE EXISTS (SELECT * FROM dz WHERE bx.b = dz.d) AND NOT EXISTS (SELECT * FROM abc WHERE bx.b = abc.a) ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -anti-join (hash) - ├── scan bx - ├── scan abc - └── filters - └── bx.b = a - -----Vertexes---- -A: -scan bx - -B: -scan abc - -----Edges---- -bx.b = a [anti] - -----Joining AB---- -A B refs [AB] [anti] - +Join Tree #1 +-------------------------------------------------------------------------------- + anti-join (hash) + ├── scan bx + ├── scan abc + └── filters + └── bx.b = a +Vertexes + A: + scan bx + B: + scan abc +Edges + bx.b = a [anti, ses=AB, tes=AB, rules=()] +Joining AB + A B [anti, refs=AB] Joins Considered: 1 -------------------------------------------------------------------------------- -----Join Tree #2---- -semi-join (hash) - ├── anti-join (hash) - │ ├── scan bx - │ ├── scan abc - │ └── filters - │ └── bx.b = a - ├── scan dz - └── filters - └── bx.b = dz.d - -----Vertexes---- -A: -scan bx - -B: -scan abc - -C: -scan dz - -----Edges---- -bx.b = a [anti] -bx.b = dz.d [semi] - -----Joining AB---- -A B refs [AB] [anti] -----Joining AC---- -A C refs [AC] [semi] -----Joining ABC---- -AC B refs [AB] [anti] -AB C refs [AC] [semi] - +Join Tree #2 +-------------------------------------------------------------------------------- + semi-join (hash) + ├── anti-join (hash) + │ ├── scan bx + │ ├── scan abc + │ └── filters + │ └── bx.b = a + ├── scan dz + └── filters + └── bx.b = dz.d +Vertexes + A: + scan bx + B: + scan abc + C: + scan dz +Edges + bx.b = a [anti, ses=AB, tes=AB, rules=()] + bx.b = dz.d [semi, ses=AC, tes=AC, rules=()] +Joining AB + A B [anti, refs=AB] +Joining AC + A C [semi, refs=AC] +Joining ABC + AC B [anti, refs=AB] + AB C [semi, refs=AC] Joins Considered: 4 -------------------------------------------------------------------------------- -----Join Tree #3---- -inner-join (hash) - ├── anti-join (hash) - │ ├── scan bx - │ ├── scan abc - │ └── filters - │ └── bx.b = a - ├── scan dz - └── filters - └── bx.b = dz.d - -----Vertexes---- -A: -scan bx - -B: -scan abc - -C: -scan dz - -----Edges---- -bx.b = a [anti] -bx.b = dz.d [inner] - -----Joining AB---- -A B refs [AB] [anti] -----Joining AC---- -A C refs [AC] [inner] -C A refs [AC] [inner] -----Joining ABC---- -AC B refs [AB] [anti] -AB C refs [AC] [inner] -C AB refs [AC] [inner] - +Join Tree #3 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── anti-join (hash) + │ ├── scan bx + │ ├── scan abc + │ └── filters + │ └── bx.b = a + ├── scan dz + └── filters + └── bx.b = dz.d +Vertexes + A: + scan bx + B: + scan abc + C: + scan dz +Edges + bx.b = a [anti, ses=AB, tes=AB, rules=()] + bx.b = dz.d [inner, ses=AC, tes=AC, rules=()] +Joining AB + A B [anti, refs=AB] +Joining AC + A C [inner, refs=AC] + C A [inner, refs=AC] +Joining ABC + AC B [anti, refs=AB] + AB C [inner, refs=AC] + C AB [inner, refs=AC] Joins Considered: 6 -------------------------------------------------------------------------------- -----Join Tree #4---- -left-join (hash) - ├── semi-join (hash) - │ ├── anti-join (hash) - │ │ ├── scan bx - │ │ ├── scan abc - │ │ └── filters - │ │ └── bx.b = a - │ ├── scan dz - │ └── filters - │ └── bx.b = dz.d - ├── scan cy - └── filters - └── bx.b = cy.c - -----Vertexes---- -A: -scan bx - -B: -scan abc - -C: -scan dz - -D: -scan cy - -----Edges---- -bx.b = a [anti] -bx.b = dz.d [semi] -bx.b = cy.c [left] - -----Joining AB---- -A B refs [AB] [anti] -----Joining AC---- -A C refs [AC] [semi] -----Joining ABC---- -AC B refs [AB] [anti] -AB C refs [AC] [semi] -----Joining AD---- -A D refs [AD] [left] -----Joining ABD---- -AD B refs [AB] [anti] -AB D refs [AD] [left] -----Joining ACD---- -AD C refs [AC] [semi] -AC D refs [AD] [left] -----Joining ABCD---- -ACD B refs [AB] [anti] -ABD C refs [AC] [semi] -ABC D refs [AD] [left] - -Joins Considered: 12 +Join Tree #4 -------------------------------------------------------------------------------- -----Final Plan---- + left-join (hash) + ├── semi-join (hash) + │ ├── anti-join (hash) + │ │ ├── scan bx + │ │ ├── scan abc + │ │ └── filters + │ │ └── bx.b = a + │ ├── scan dz + │ └── filters + │ └── bx.b = dz.d + ├── scan cy + └── filters + └── bx.b = cy.c +Vertexes + A: + scan bx + B: + scan abc + C: + scan dz + D: + scan cy +Edges + bx.b = a [anti, ses=AB, tes=AB, rules=()] + bx.b = dz.d [semi, ses=AC, tes=AC, rules=()] + bx.b = cy.c [left, ses=AD, tes=AD, rules=()] +Joining AB + A B [anti, refs=AB] +Joining AC + A C [semi, refs=AC] +Joining ABC + AC B [anti, refs=AB] + AB C [semi, refs=AC] +Joining AD + A D [left, refs=AD] +Joining ABD + AD B [anti, refs=AB] + AB D [left, refs=AD] +Joining ACD + AD C [semi, refs=AC] + AC D [left, refs=AD] +Joining ABCD + ACD B [anti, refs=AB] + ABD C [semi, refs=AC] + ABC D [left, refs=AD] +Joins Considered: 12 +================================================================================ +Final Plan +================================================================================ semi-join (lookup dz) ├── lookup columns are key ├── left-join (lookup cy) @@ -1977,9 +1844,6 @@ semi-join (lookup dz) │ │ └── filters (true) │ └── filters (true) └── filters (true) --------------------------------------------------------------------------------- ----- ----- # Join tree with only cross joins. reorderjoins format=hide-all @@ -1987,63 +1851,54 @@ SELECT * FROM bx INNER JOIN cy ON True INNER JOIN dz ON True ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -inner-join (cross) - ├── scan bx - ├── scan cy - └── filters (true) - -----Vertexes---- -A: -scan bx - -B: -scan cy - -----Edges---- -cross [inner] - -----Joining AB---- -A B refs [] [inner] -B A refs [] [inner] - +Join Tree #1 +-------------------------------------------------------------------------------- + inner-join (cross) + ├── scan bx + ├── scan cy + └── filters (true) +Vertexes + A: + scan bx + B: + scan cy +Edges + cross [inner, ses=, tes=AB, rules=()] +Joining AB + A B [inner, refs=] + B A [inner, refs=] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #2---- -inner-join (cross) - ├── inner-join (cross) - │ ├── scan bx - │ ├── scan cy - │ └── filters (true) - ├── scan dz - └── filters (true) - -----Vertexes---- -A: -scan bx - -B: -scan cy - -C: -scan dz - -----Edges---- -cross [inner] -cross [inner] - -----Joining AB---- -A B refs [] [inner] -B A refs [] [inner] -----Joining ABC---- -AB C refs [] [inner] -C AB refs [] [inner] - -Joins Considered: 4 +Join Tree #2 -------------------------------------------------------------------------------- -----Final Plan---- + inner-join (cross) + ├── inner-join (cross) + │ ├── scan bx + │ ├── scan cy + │ └── filters (true) + ├── scan dz + └── filters (true) +Vertexes + A: + scan bx + B: + scan cy + C: + scan dz +Edges + cross [inner, ses=, tes=AB, rules=()] + cross [inner, ses=, tes=ABC, rules=()] +Joining AB + A B [inner, refs=] + B A [inner, refs=] +Joining ABC + AB C [inner, refs=] + C AB [inner, refs=] +Joins Considered: 4 +================================================================================ +Final Plan +================================================================================ inner-join (cross) ├── inner-join (cross) │ ├── scan bx @@ -2051,9 +1906,6 @@ inner-join (cross) │ └── filters (true) ├── scan dz └── filters (true) --------------------------------------------------------------------------------- ----- ----- reorderjoins format=hide-all SELECT * @@ -2061,71 +1913,62 @@ FROM bx FULL JOIN cy ON b = c FULL JOIN dz ON y = z ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -full-join (hash) - ├── scan bx - ├── scan cy - └── filters - └── b = c - -----Vertexes---- -A: -scan bx - -B: -scan cy - -----Edges---- -b = c [full] - -----Joining AB---- -A B refs [AB] [full] -B A refs [AB] [full] - +Join Tree #1 +-------------------------------------------------------------------------------- + full-join (hash) + ├── scan bx + ├── scan cy + └── filters + └── b = c +Vertexes + A: + scan bx + B: + scan cy +Edges + b = c [full, ses=AB, tes=AB, rules=()] +Joining AB + A B [full, refs=AB] + B A [full, refs=AB] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #2---- -full-join (hash) - ├── full-join (hash) - │ ├── scan bx - │ ├── scan cy - │ └── filters - │ └── b = c - ├── scan dz - └── filters - └── y = z - -----Vertexes---- -A: -scan bx - -B: -scan cy - -C: -scan dz - -----Edges---- -b = c [full] -y = z [full] - -----Joining AB---- -A B refs [AB] [full] -B A refs [AB] [full] -----Joining BC---- -B C refs [BC] [full] -C B refs [BC] [full] -----Joining ABC---- -A BC refs [AB] [full] -BC A refs [AB] [full] -AB C refs [BC] [full] -C AB refs [BC] [full] - -Joins Considered: 8 +Join Tree #2 -------------------------------------------------------------------------------- -----Final Plan---- + full-join (hash) + ├── full-join (hash) + │ ├── scan bx + │ ├── scan cy + │ └── filters + │ └── b = c + ├── scan dz + └── filters + └── y = z +Vertexes + A: + scan bx + B: + scan cy + C: + scan dz +Edges + b = c [full, ses=AB, tes=AB, rules=()] + y = z [full, ses=BC, tes=BC, rules=()] +Joining AB + A B [full, refs=AB] + B A [full, refs=AB] +Joining BC + B C [full, refs=BC] + C B [full, refs=BC] +Joining ABC + A BC [full, refs=AB] + BC A [full, refs=AB] + AB C [full, refs=BC] + C AB [full, refs=BC] +Joins Considered: 8 +================================================================================ +Final Plan +================================================================================ full-join (hash) ├── full-join (merge) │ ├── scan bx @@ -2134,9 +1977,6 @@ full-join (hash) ├── scan dz └── filters └── y = z --------------------------------------------------------------------------------- ----- ----- # Iteratively reorder subtrees of up to size 2. reorderjoins join-limit=2 format=hide-all @@ -2146,236 +1986,208 @@ LEFT JOIN abc AS a3 ON a2.b = a3.b INNER JOIN abc AS a4 ON a3.a = a4.a WHERE EXISTS (SELECT * FROM abc AS a5 WHERE a2.c = a5.c) ---- ----- -------------------------------------------------------------------------------- -----Join Tree #1---- -semi-join (hash) - ├── scan abc [as=a2] - ├── scan abc [as=a5] - └── filters - └── a2.c = a5.c - -----Vertexes---- -A: -scan abc [as=a2] - -B: -scan abc [as=a5] - -----Edges---- -a2.c = a5.c [semi] - -----Joining AB---- -A B refs [AB] [semi] - +Join Tree #1 +-------------------------------------------------------------------------------- + semi-join (hash) + ├── scan abc [as=a2] + ├── scan abc [as=a5] + └── filters + └── a2.c = a5.c +Vertexes + A: + scan abc [as=a2] + B: + scan abc [as=a5] +Edges + a2.c = a5.c [semi, ses=AB, tes=AB, rules=()] +Joining AB + A B [semi, refs=AB] Joins Considered: 1 -------------------------------------------------------------------------------- -----Join Tree #2---- -inner-join (hash) - ├── scan abc [as=a2] - ├── distinct-on - │ └── scan abc [as=a5] - └── filters - └── a2.c = a5.c - -----Vertexes---- -A: -scan abc [as=a2] - -B: -distinct-on - └── scan abc [as=a5] - -----Edges---- -a2.c = a5.c [inner] - -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] - +Join Tree #2 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── scan abc [as=a2] + ├── distinct-on + │ └── scan abc [as=a5] + └── filters + └── a2.c = a5.c +Vertexes + A: + scan abc [as=a2] + B: + distinct-on + └── scan abc [as=a5] +Edges + a2.c = a5.c [inner, ses=AB, tes=AB, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] Joins Considered: 2 -------------------------------------------------------------------------------- -----Join Tree #3---- -inner-join (hash) - ├── scan abc [as=a1] - ├── semi-join (hash) - │ ├── scan abc [as=a2] - │ ├── scan abc [as=a5] - │ └── filters - │ └── a2.c = a5.c - └── filters - └── a1.a = a2.a - -----Vertexes---- -C: -scan abc [as=a1] - -A: -scan abc [as=a2] - -B: -scan abc [as=a5] - -----Edges---- -a2.c = a5.c [semi] -a1.a = a2.a [inner] - -----Joining CA---- -C A refs [CA] [inner] -A C refs [CA] [inner] -----Joining AB---- -A B refs [AB] [semi] -----Joining CAB---- -C AB refs [CA] [inner] -AB C refs [CA] [inner] -CA B refs [AB] [semi] - +Join Tree #3 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── scan abc [as=a1] + ├── semi-join (hash) + │ ├── scan abc [as=a2] + │ ├── scan abc [as=a5] + │ └── filters + │ └── a2.c = a5.c + └── filters + └── a1.a = a2.a +Vertexes + C: + scan abc [as=a1] + A: + scan abc [as=a2] + B: + scan abc [as=a5] +Edges + a2.c = a5.c [semi, ses=AB, tes=AB, rules=()] + a1.a = a2.a [inner, ses=CA, tes=CA, rules=()] +Joining CA + C A [inner, refs=CA] + A C [inner, refs=CA] +Joining AB + A B [semi, refs=AB] +Joining CAB + C AB [inner, refs=CA] + AB C [inner, refs=CA] + CA B [semi, refs=AB] Joins Considered: 6 -------------------------------------------------------------------------------- -----Join Tree #4---- -inner-join (hash) - ├── scan abc [as=a1] - ├── inner-join (hash) - │ ├── scan abc [as=a2] - │ ├── distinct-on - │ │ └── scan abc [as=a5] - │ └── filters - │ └── a2.c = a5.c - └── filters - └── a1.a = a2.a - -----Vertexes---- -C: -scan abc [as=a1] - -A: -scan abc [as=a2] - -B: -distinct-on - └── scan abc [as=a5] - -----Edges---- -a2.c = a5.c [inner] -a1.a = a2.a [inner] - -----Joining CA---- -C A refs [CA] [inner] -A C refs [CA] [inner] -----Joining AB---- -A B refs [AB] [inner] -B A refs [AB] [inner] -----Joining CAB---- -C AB refs [CA] [inner] -AB C refs [CA] [inner] -CA B refs [AB] [inner] -B CA refs [AB] [inner] - -Joins Considered: 8 +Join Tree #4 -------------------------------------------------------------------------------- -----Join Tree #5---- -inner-join (hash) - ├── inner-join (hash) - │ ├── scan abc [as=a1] - │ ├── semi-join (hash) - │ │ ├── scan abc [as=a2] - │ │ ├── scan abc [as=a5] - │ │ └── filters - │ │ └── a2.c = a5.c - │ └── filters - │ └── a1.a = a2.a - ├── scan abc [as=a3] - └── filters - └── a2.b = a3.b - -----Vertexes---- -C: -scan abc [as=a1] - -A: -semi-join (hash) - ├── scan abc [as=a2] - ├── scan abc [as=a5] - └── filters - └── a2.c = a5.c - -D: -scan abc [as=a3] - -----Edges---- -a1.a = a2.a [inner] -a2.b = a3.b [inner] - -----Joining CA---- -C A refs [CA] [inner] -A C refs [CA] [inner] -----Joining AD---- -A D refs [AD] [inner] -D A refs [AD] [inner] -----Joining CAD---- -C AD refs [CA] [inner] -AD C refs [CA] [inner] -CA D refs [AD] [inner] -D CA refs [AD] [inner] - + inner-join (hash) + ├── scan abc [as=a1] + ├── inner-join (hash) + │ ├── scan abc [as=a2] + │ ├── distinct-on + │ │ └── scan abc [as=a5] + │ └── filters + │ └── a2.c = a5.c + └── filters + └── a1.a = a2.a +Vertexes + C: + scan abc [as=a1] + A: + scan abc [as=a2] + B: + distinct-on + └── scan abc [as=a5] +Edges + a2.c = a5.c [inner, ses=AB, tes=AB, rules=()] + a1.a = a2.a [inner, ses=CA, tes=CA, rules=()] +Joining CA + C A [inner, refs=CA] + A C [inner, refs=CA] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining CAB + C AB [inner, refs=CA] + AB C [inner, refs=CA] + CA B [inner, refs=AB] + B CA [inner, refs=AB] Joins Considered: 8 -------------------------------------------------------------------------------- -----Join Tree #6---- -inner-join (hash) - ├── inner-join (hash) - │ ├── inner-join (hash) - │ │ ├── scan abc [as=a1] - │ │ ├── semi-join (hash) - │ │ │ ├── scan abc [as=a2] - │ │ │ ├── scan abc [as=a5] - │ │ │ └── filters - │ │ │ └── a2.c = a5.c - │ │ └── filters - │ │ └── a1.a = a2.a - │ ├── scan abc [as=a3] - │ └── filters - │ └── a2.b = a3.b - ├── scan abc [as=a4] - └── filters - └── a3.a = a4.a - -----Vertexes---- -C: -inner-join (hash) - ├── scan abc [as=a1] - ├── semi-join (hash) - │ ├── scan abc [as=a2] - │ ├── scan abc [as=a5] - │ └── filters - │ └── a2.c = a5.c - └── filters - └── a1.a = a2.a - -D: -scan abc [as=a3] - -E: -scan abc [as=a4] - -----Edges---- -a2.b = a3.b [inner] -a3.a = a4.a [inner] - -----Joining CD---- -C D refs [CD] [inner] -D C refs [CD] [inner] -----Joining DE---- -D E refs [DE] [inner] -E D refs [DE] [inner] -----Joining CDE---- -C DE refs [CD] [inner] -DE C refs [CD] [inner] -CD E refs [DE] [inner] -E CD refs [DE] [inner] - +Join Tree #5 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── inner-join (hash) + │ ├── scan abc [as=a1] + │ ├── semi-join (hash) + │ │ ├── scan abc [as=a2] + │ │ ├── scan abc [as=a5] + │ │ └── filters + │ │ └── a2.c = a5.c + │ └── filters + │ └── a1.a = a2.a + ├── scan abc [as=a3] + └── filters + └── a2.b = a3.b +Vertexes + C: + scan abc [as=a1] + A: + semi-join (hash) + ├── scan abc [as=a2] + ├── scan abc [as=a5] + └── filters + └── a2.c = a5.c + D: + scan abc [as=a3] +Edges + a1.a = a2.a [inner, ses=CA, tes=CA, rules=()] + a2.b = a3.b [inner, ses=AD, tes=AD, rules=()] +Joining CA + C A [inner, refs=CA] + A C [inner, refs=CA] +Joining AD + A D [inner, refs=AD] + D A [inner, refs=AD] +Joining CAD + C AD [inner, refs=CA] + AD C [inner, refs=CA] + CA D [inner, refs=AD] + D CA [inner, refs=AD] Joins Considered: 8 -------------------------------------------------------------------------------- -----Final Plan---- +Join Tree #6 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── inner-join (hash) + │ ├── inner-join (hash) + │ │ ├── scan abc [as=a1] + │ │ ├── semi-join (hash) + │ │ │ ├── scan abc [as=a2] + │ │ │ ├── scan abc [as=a5] + │ │ │ └── filters + │ │ │ └── a2.c = a5.c + │ │ └── filters + │ │ └── a1.a = a2.a + │ ├── scan abc [as=a3] + │ └── filters + │ └── a2.b = a3.b + ├── scan abc [as=a4] + └── filters + └── a3.a = a4.a +Vertexes + C: + inner-join (hash) + ├── scan abc [as=a1] + ├── semi-join (hash) + │ ├── scan abc [as=a2] + │ ├── scan abc [as=a5] + │ └── filters + │ └── a2.c = a5.c + └── filters + └── a1.a = a2.a + D: + scan abc [as=a3] + E: + scan abc [as=a4] +Edges + a2.b = a3.b [inner, ses=CD, tes=CD, rules=()] + a3.a = a4.a [inner, ses=DE, tes=DE, rules=()] +Joining CD + C D [inner, refs=CD] + D C [inner, refs=CD] +Joining DE + D E [inner, refs=DE] + E D [inner, refs=DE] +Joining CDE + C DE [inner, refs=CD] + DE C [inner, refs=CD] + CD E [inner, refs=DE] + E CD [inner, refs=DE] +Joins Considered: 8 +================================================================================ +Final Plan +================================================================================ inner-join (hash) ├── project │ └── inner-join (hash) @@ -2393,9 +2205,247 @@ inner-join (hash) │ └── filters (true) └── filters └── a2.b = a3.b --------------------------------------------------------------------------------- ----- + +# Reorder with a conflict rule. +reorderjoins +SELECT * FROM +( + SELECT * FROM abc + LEFT JOIN bx ON a = x + WHERE EXISTS (SELECT * FROM cy WHERE x = y) +) +INNER JOIN dz ON a = z ---- +-------------------------------------------------------------------------------- +Join Tree #1 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── scan abc + ├── select + │ ├── scan bx + │ └── filters + │ └── x IS NOT NULL + └── filters + └── a = x +Vertexes + A: + scan abc + B: + select + ├── scan bx + └── filters + └── x IS NOT NULL +Edges + a = x [inner, ses=AB, tes=AB, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joins Considered: 2 +-------------------------------------------------------------------------------- +Join Tree #2 +-------------------------------------------------------------------------------- + semi-join (hash) + ├── inner-join (hash) + │ ├── scan abc + │ ├── select + │ │ ├── scan bx + │ │ └── filters + │ │ └── x IS NOT NULL + │ └── filters + │ └── a = x + ├── scan cy + └── filters + └── x = y +Vertexes + A: + scan abc + B: + select + ├── scan bx + └── filters + └── x IS NOT NULL + C: + scan cy +Edges + a = x [inner, ses=AB, tes=AB, rules=()] + x = y [semi, ses=BC, tes=BC, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining BC + B C [semi, refs=BC] +Joining ABC + A BC [inner, refs=AB] + BC A [inner, refs=AB] + AB C [semi, refs=BC] +Joins Considered: 6 +-------------------------------------------------------------------------------- +Join Tree #3 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── inner-join (hash) + │ ├── scan abc + │ ├── select + │ │ ├── scan bx + │ │ └── filters + │ │ └── x IS NOT NULL + │ └── filters + │ └── a = x + ├── distinct-on + │ └── scan cy + └── filters + └── x = y +Vertexes + A: + scan abc + B: + select + ├── scan bx + └── filters + └── x IS NOT NULL + C: + distinct-on + └── scan cy +Edges + a = x [inner, ses=AB, tes=AB, rules=()] + x = y [inner, ses=BC, tes=BC, rules=()] + a = y [inner, ses=AC, tes=AC, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining AC + A C [inner, refs=AC] + C A [inner, refs=AC] +Joining BC + B C [inner, refs=BC] + C B [inner, refs=BC] +Joining ABC + A BC [inner, refs=AB] + BC A [inner, refs=AB] + B AC [inner, refs=AB] + AC B [inner, refs=AB] + AB C [inner, refs=BC] + C AB [inner, refs=BC] +Joins Considered: 12 +-------------------------------------------------------------------------------- +Join Tree #4 +-------------------------------------------------------------------------------- + inner-join (hash) + ├── semi-join (hash) + │ ├── inner-join (hash) + │ │ ├── scan abc + │ │ ├── select + │ │ │ ├── scan bx + │ │ │ └── filters + │ │ │ └── x IS NOT NULL + │ │ └── filters + │ │ └── a = x + │ ├── scan cy + │ └── filters + │ └── x = y + ├── scan dz + └── filters + └── a = z +Vertexes + A: + scan abc + B: + select + ├── scan bx + └── filters + └── x IS NOT NULL + C: + scan cy + D: + scan dz +Edges + a = x [inner, ses=AB, tes=AB, rules=()] + x = y [semi, ses=BC, tes=BC, rules=()] + a = z [inner, ses=AD, tes=AD, rules=(C->B)] + x = z [inner, ses=BD, tes=BD, rules=()] +Joining AB + A B [inner, refs=AB] + B A [inner, refs=AB] +Joining BC + B C [semi, refs=BC] +Joining ABC + A BC [inner, refs=AB] + BC A [inner, refs=AB] + AB C [semi, refs=BC] +Joining AD + A D [inner, refs=AD] + D A [inner, refs=AD] +Joining BD + B D [inner, refs=BD] + D B [inner, refs=BD] +Joining ABD + A BD [inner, refs=AB] + BD A [inner, refs=AB] + B AD [inner, refs=AB] + AD B [inner, refs=AB] + AB D [inner, refs=AD] + D AB [inner, refs=AD] +Joining BCD + BD C [semi, refs=BC] + BC D [inner, refs=BD] + D BC [inner, refs=BD] +Joining ABCD + A BCD [inner, refs=AB] + BCD A [inner, refs=AB] + ABD C [semi, refs=BC] + BC AD [inner, refs=AB] + AD BC [inner, refs=AB] + ABC D [inner, refs=AD] + D ABC [inner, refs=AD] +Joins Considered: 26 +================================================================================ +Final Plan +================================================================================ +inner-join (hash) + ├── columns: a:1!null b:2 c:3 d:4 b:7!null x:8!null d:15!null z:16!null + ├── key: (7,15) + ├── fd: (1)-->(2-4), (7)-->(8), (1)==(8,16), (8)==(1,16), (15)-->(16), (16)==(1,8) + ├── scan dz + │ ├── columns: dz.d:15!null z:16 + │ ├── key: (15) + │ └── fd: (15)-->(16) + ├── project + │ ├── columns: a:1!null abc.b:2 abc.c:3 abc.d:4 bx.b:7!null x:8!null + │ ├── key: (7) + │ ├── fd: (1)-->(2-4), (7)-->(8), (1)==(8), (8)==(1) + │ └── inner-join (hash) + │ ├── columns: a:1!null abc.b:2 abc.c:3 abc.d:4 bx.b:7!null x:8!null y:12!null + │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ ├── key: (7) + │ ├── fd: (1)-->(2-4), (7)-->(8), (1)==(8,12), (8)==(1,12), (12)==(1,8) + │ ├── select + │ │ ├── columns: bx.b:7!null x:8!null + │ │ ├── key: (7) + │ │ ├── fd: (7)-->(8) + │ │ ├── scan bx + │ │ │ ├── columns: bx.b:7!null x:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) + │ │ └── filters + │ │ └── x:8 IS NOT NULL [outer=(8), constraints=(/8: (/NULL - ]; tight)] + │ ├── inner-join (lookup abc) + │ │ ├── columns: a:1!null abc.b:2 abc.c:3 abc.d:4 y:12!null + │ │ ├── key columns: [12] = [1] + │ │ ├── lookup columns are key + │ │ ├── key: (12) + │ │ ├── fd: (1)-->(2-4), (1)==(12), (12)==(1) + │ │ ├── distinct-on + │ │ │ ├── columns: y:12 + │ │ │ ├── grouping columns: y:12 + │ │ │ ├── key: (12) + │ │ │ └── scan cy + │ │ │ └── columns: y:12 + │ │ └── filters (true) + │ └── filters + │ └── a:1 = x:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + └── filters + └── a:1 = z:16 [outer=(1,16), constraints=(/1: (/NULL - ]; /16: (/NULL - ]), fd=(1)==(16), (16)==(1)] + # Regression test for #59076. Do not reorder on the inner join produced by # CommuteSemiJoin when it matches on an already-reordered semi join because @@ -2408,3 +2458,191 @@ AND EXISTS (SELECT 1 FROM abc WHERE a = y) ---- Rules Applied: 148 Groups Added: 80 + + +# Regression test for #76522. Do not produce query plans where some of the +# original filters have been omitted. + +exec-ddl +CREATE TABLE t76522_1 ( + a INT NOT NULL, + b INT NOT NULL, + PRIMARY KEY (a ASC, b ASC) +) +---- + +exec-ddl +CREATE TABLE t76522_2 ( + a INT NOT NULL, + c INT, + should_not_be_eliminated INT, + PRIMARY KEY (a ASC) +) +---- + +exec-ddl +CREATE TABLE t76522_3 ( + a INT NOT NULL, + d INT NOT NULL, + f INT, + g INT, + PRIMARY KEY (a ASC, d ASC) +) +---- + +exec-ddl +CREATE TABLE t76522_4 ( + e INT NOT NULL, + f INT, + g INT, + PRIMARY KEY (e ASC) +) +---- + +exec-ddl +CREATE TABLE t76522_5 ( + h INT NOT NULL, + f INT NOT NULL, + g INT NOT NULL, + b INT, + should_not_be_eliminated INT, + c INT, + PRIMARY KEY (h ASC, f ASC, g ASC) +) +---- + +# Give t76522_1 many rows where a has many distincts. +exec-ddl +ALTER TABLE t76522_1 INJECT STATISTICS '[ + { + "columns": [ + "a" + ], + "created_at": "2022-01-17 12:51:38.433911", + "distinct_count": 9161427, + "null_count": 0, + "row_count": 44484238 + } +]' +---- + +# Give t76522_2 many rows where a has many distincts. +exec-ddl +ALTER TABLE t76522_2 INJECT STATISTICS '[ + { + "columns": [ + "a" + ], + "created_at": "2022-01-17 12:51:38.433911", + "distinct_count": 17014025, + "null_count": 0, + "row_count": 17024553 + } +]' +---- + +# Give t76522_3 many rows where a has many distincts. +exec-ddl +ALTER TABLE t76522_3 INJECT STATISTICS '[ + { + "columns": [ + "a" + ], + "created_at": "2022-01-17 12:51:38.433911", + "distinct_count": 17187349, + "null_count": 0, + "row_count": 18138540 + } +]' +---- + +# Give t76522_4 many rows where e has many distincts. +exec-ddl +ALTER TABLE t76522_4 INJECT STATISTICS '[ + { + "columns": [ + "e" + ], + "created_at": "2022-01-17 12:51:38.433911", + "distinct_count": 346919, + "null_count": 0, + "row_count": 346109 + } +]'; +---- + +# Give t5 few rows. +exec-ddl +ALTER TABLE t76522_5 INJECT STATISTICS '[ + { + "columns": [ + "h" + ], + "created_at": "2022-01-17 12:51:38.433911", + "distinct_count": 119, + "null_count": 0, + "row_count": 119 + } +]' +---- + +# Prior to the fix, these filters were missing from the query plan: +# +# t5.c = t2.c +# t2.should_not_be_eliminated = t5.should_not_be_eliminated +# +opt +SELECT + t2.a +FROM + t76522_1 AS t1 + INNER JOIN t76522_2 AS t2 ON t1.a = t2.a + INNER JOIN t76522_3 AS t3 ON t1.a = t3.a + INNER JOIN t76522_4 AS t4 ON t3.d = t4.e + INNER JOIN t76522_5 AS t5 ON + t4.f = t5.f + AND t4.g = t5.g + AND t5.b = t1.b + AND t5.c = t2.c +WHERE + t1.a = 123456 AND t2.should_not_be_eliminated = t5.should_not_be_eliminated; +---- +project + ├── columns: a:5!null + ├── fd: ()-->(5) + └── inner-join (lookup t76522_1 [as=t1]) + ├── columns: t1.a:1!null t1.b:2!null t2.a:5!null t2.c:6!null t2.should_not_be_eliminated:7!null t3.a:10!null d:11!null e:16!null t4.f:17!null t4.g:18!null t5.f:22!null t5.g:23!null t5.b:24!null t5.should_not_be_eliminated:25!null t5.c:26!null + ├── key columns: [5 24] = [1 2] + ├── lookup columns are key + ├── fd: ()-->(1,5-7,10,25,26), (1)==(5,10), (5)==(1,10), (10)==(1,5), (16)-->(17,18), (11)==(16), (16)==(11), (17)==(22), (22)==(17), (18)==(23), (23)==(18), (2)==(24), (24)==(2), (6)==(26), (26)==(6), (7)==(25), (25)==(7) + ├── inner-join (lookup t76522_2 [as=t2]) + │ ├── columns: t2.a:5!null t2.c:6!null t2.should_not_be_eliminated:7!null t3.a:10!null d:11!null e:16!null t4.f:17!null t4.g:18!null t5.f:22!null t5.g:23!null t5.b:24 t5.should_not_be_eliminated:25!null t5.c:26!null + │ ├── key columns: [10] = [5] + │ ├── lookup columns are key + │ ├── fd: ()-->(5-7,10,25,26), (16)-->(17,18), (17)==(22), (22)==(17), (18)==(23), (23)==(18), (11)==(16), (16)==(11), (6)==(26), (26)==(6), (7)==(25), (25)==(7), (5)==(10), (10)==(5) + │ ├── inner-join (hash) + │ │ ├── columns: t3.a:10!null d:11!null e:16!null t4.f:17!null t4.g:18!null t5.f:22!null t5.g:23!null t5.b:24 t5.should_not_be_eliminated:25 t5.c:26 + │ │ ├── fd: ()-->(10), (16)-->(17,18), (17)==(22), (22)==(17), (18)==(23), (23)==(18), (11)==(16), (16)==(11) + │ │ ├── scan t76522_5 [as=t5] + │ │ │ └── columns: t5.f:22!null t5.g:23!null t5.b:24 t5.should_not_be_eliminated:25 t5.c:26 + │ │ ├── inner-join (lookup t76522_4 [as=t4]) + │ │ │ ├── columns: t3.a:10!null d:11!null e:16!null t4.f:17 t4.g:18 + │ │ │ ├── key columns: [11] = [16] + │ │ │ ├── lookup columns are key + │ │ │ ├── key: (16) + │ │ │ ├── fd: ()-->(10), (16)-->(17,18), (11)==(16), (16)==(11) + │ │ │ ├── scan t76522_3 [as=t3] + │ │ │ │ ├── columns: t3.a:10!null d:11!null + │ │ │ │ ├── constraint: /10/11: [/123456 - /123456] + │ │ │ │ ├── key: (11) + │ │ │ │ └── fd: ()-->(10) + │ │ │ └── filters (true) + │ │ └── filters + │ │ ├── t4.f:17 = t5.f:22 [outer=(17,22), constraints=(/17: (/NULL - ]; /22: (/NULL - ]), fd=(17)==(22), (22)==(17)] + │ │ └── t4.g:18 = t5.g:23 [outer=(18,23), constraints=(/18: (/NULL - ]; /23: (/NULL - ]), fd=(18)==(23), (23)==(18)] + │ └── filters + │ ├── t5.c:26 = t2.c:6 [outer=(6,26), constraints=(/6: (/NULL - ]; /26: (/NULL - ]), fd=(6)==(26), (26)==(6)] + │ ├── t2.should_not_be_eliminated:7 = t5.should_not_be_eliminated:25 [outer=(7,25), constraints=(/7: (/NULL - ]; /25: (/NULL - ]), fd=(7)==(25), (25)==(7)] + │ └── t2.a:5 = 123456 [outer=(5), constraints=(/5: [/123456 - /123456]; tight), fd=()-->(5)] + └── filters + └── t1.a:1 = 123456 [outer=(1), constraints=(/1: [/123456 - /123456]; tight), fd=()-->(1)] diff --git a/pkg/sql/randgen/BUILD.bazel b/pkg/sql/randgen/BUILD.bazel index 42ec357de1b9..411d1c94de0b 100644 --- a/pkg/sql/randgen/BUILD.bazel +++ b/pkg/sql/randgen/BUILD.bazel @@ -51,7 +51,23 @@ go_library( go_test( name = "randgen_test", - srcs = ["mutator_test.go"], + srcs = [ + "main_test.go", + "mutator_test.go", + "schema_test.go", + ], embed = [":randgen"], - deps = ["//pkg/util/randutil"], + deps = [ + "//pkg/base", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/sql/sem/tree", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/randutil", + "@com_github_stretchr_testify//require", + ], ) diff --git a/pkg/sql/randgen/main_test.go b/pkg/sql/randgen/main_test.go new file mode 100644 index 000000000000..722cb9833e3d --- /dev/null +++ b/pkg/sql/randgen/main_test.go @@ -0,0 +1,33 @@ +// 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. + +package randgen + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +//go:generate ../../util/leaktest/add-leaktest.sh *_test.go + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/sql/randgen/mutator_test.go b/pkg/sql/randgen/mutator_test.go index 152f7e2edc78..77eb31061237 100644 --- a/pkg/sql/randgen/mutator_test.go +++ b/pkg/sql/randgen/mutator_test.go @@ -14,10 +14,12 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" ) func TestPostgresMutator(t *testing.T) { + defer leaktest.AfterTest(t)() q := ` CREATE TABLE t (s STRING FAMILY fam1, b BYTES, FAMILY fam2 (b), PRIMARY KEY (s ASC, b DESC), INDEX (s) STORING (b)) PARTITION BY LIST (s) diff --git a/pkg/sql/randgen/schema.go b/pkg/sql/randgen/schema.go index a5c73399cc49..849da14ef6f6 100644 --- a/pkg/sql/randgen/schema.go +++ b/pkg/sql/randgen/schema.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -186,6 +187,138 @@ func RandCreateTableWithColumnIndexNumberGenerator( return res[0].(*tree.CreateTable) } +func parseCreateStatement(createStmtSQL string) (*tree.CreateTable, error) { + var p parser.Parser + stmts, err := p.Parse(createStmtSQL) + if err != nil { + return nil, err + } + if len(stmts) != 1 { + return nil, errors.Errorf("parsed CreateStatement string yielded more than one parsed statment") + } + tableStmt, ok := stmts[0].AST.(*tree.CreateTable) + if !ok { + return nil, errors.Errorf("AST could not be cast to *tree.CreateTable") + } + return tableStmt, nil +} + +// generateInsertStmtVals generates random data for a string builder thats +// used after the VALUES keyword in an INSERT statement. +func generateInsertStmtVals(rng *rand.Rand, colTypes []*types.T, nullable []bool) string { + var valBuilder strings.Builder + valBuilder.WriteString("(") + comma := "" + for j := 0; j < len(colTypes); j++ { + valBuilder.WriteString(comma) + var d tree.Datum + if rand.Intn(10) < 4 { + // 40% of the time, use a corner case value + d = randInterestingDatum(rng, colTypes[j]) + } + if colTypes[j] == types.RegType { + // RandDatum is naive to the constraint that a RegType < len(types.OidToType), + // at least before linking and user defined types are added. + d = tree.NewDOid(tree.DInt(rand.Intn(len(types.OidToType)))) + } + if d == nil { + d = RandDatum(rng, colTypes[j], nullable[j]) + } + valBuilder.WriteString(tree.AsStringWithFlags(d, tree.FmtParsable)) + comma = ", " + } + valBuilder.WriteString(")") + return valBuilder.String() +} + +// TODO(butler): develop new helper function PopulateDatabaseWithRandData which calls +// PopulateTableWithRandData on each table in the order of the fk +// dependency graph. + +// PopulateTableWithRandData populates the provided table by executing exactly +// `numInserts` statements. numRowsInserted <= numInserts because inserting into +// an arbitrary table can fail for reasons which include: +// - UNIQUE or CHECK constraint violation. RandDatum is naive to these constraints. +// - Out of range error for a computed INT2 or INT4 column. +// +// If numRowsInserted == 0, PopulateTableWithRandomData or RandDatum couldn't +// handle this table's schema. Consider increasing numInserts or filing a bug. +func PopulateTableWithRandData( + rng *rand.Rand, db *gosql.DB, tableName string, numInserts int, +) (numRowsInserted int, err error) { + var createStmtSQL string + res := db.QueryRow(fmt.Sprintf("SELECT create_statement FROM [SHOW CREATE TABLE %s]", tableName)) + err = res.Scan(&createStmtSQL) + if err != nil { + return 0, errors.Wrapf(err, "table does not exist in db") + } + createStmt, err := parseCreateStatement(createStmtSQL) + if err != nil { + return 0, errors.Wrapf(err, "failed to determine table schema") + } + + // Find columns subject to a foreign key constraint + var hasFK = map[string]bool{} + for _, def := range createStmt.Defs { + if fk, ok := def.(*tree.ForeignKeyConstraintTableDef); ok { + for _, col := range fk.FromCols { + hasFK[col.String()] = true + } + } + } + + // Populate helper objects for insert statement creation and error out if a + // column's constraints will make it impossible to execute random insert + // statements. + + colTypes := make([]*types.T, 0) + nullable := make([]bool, 0) + var colNameBuilder strings.Builder + comma := "" + for _, def := range createStmt.Defs { + if col, ok := def.(*tree.ColumnTableDef); ok { + if _, ok := hasFK[col.Name.String()]; ok { + // Given that this function only populates an individual table without + // considering other tables in the database, populating a column with a + // foreign key reference with actual data can be nearly impossible. To + // make inserts pass more frequently, this function skips populating + // columns with a foreign key reference. Sadly, if these columns with + // FKs are also NOT NULL, 0 rows will get inserted. + + // TODO(butler): get the unique values from each foreign key reference and + // populate the column by sampling the FK's unique values. + if col.Nullable.Nullability == tree.Null { + continue + } + } + if col.Computed.Computed || col.Hidden { + // Cannot insert values into hidden or computed columns, so skip adding + // them to the list of columns to insert data into. + continue + } + colTypes = append(colTypes, tree.MustBeStaticallyKnownType(col.Type.(*types.T))) + nullable = append(nullable, col.Nullable.Nullability == tree.Null) + + colNameBuilder.WriteString(comma) + colNameBuilder.WriteString(col.Name.String()) + comma = ", " + } + } + + for i := 0; i < numInserts; i++ { + insertVals := generateInsertStmtVals(rng, colTypes, nullable) + insertStmt := fmt.Sprintf("INSERT INTO %s (%s) VALUES %s;", + tableName, + colNameBuilder.String(), + insertVals) + _, err := db.Exec(insertStmt) + if err == nil { + numRowsInserted++ + } + } + return numRowsInserted, nil +} + // GenerateRandInterestingTable takes a gosql.DB connection and creates // a table with all the types in randInterestingDatums and rows of the // interesting datums. diff --git a/pkg/sql/randgen/schema_test.go b/pkg/sql/randgen/schema_test.go new file mode 100644 index 000000000000..c7c5ab6ee17c --- /dev/null +++ b/pkg/sql/randgen/schema_test.go @@ -0,0 +1,76 @@ +// Copyright 2022 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 randgen + +import ( + "context" + "fmt" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/stretchr/testify/require" +) + +// TestPopulateTableWithRandData generates some random tables and passes if it +// at least one of those tables will be successfully populated. +func TestPopulateTableWithRandData(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + s, dbConn, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + rng, _ := randutil.NewTestRand() + + sqlDB := sqlutils.MakeSQLRunner(dbConn) + sqlDB.Exec(t, "CREATE DATABASE rand") + + // Turn off auto stats collection to prevent out of memory errors on stress tests + sqlDB.Exec(t, "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false") + + tablePrefix := "table" + numTables := 10 + + stmts := RandCreateTables(rng, tablePrefix, numTables, + PartialIndexMutator, + ForeignKeyMutator, + ) + + var sb strings.Builder + for _, stmt := range stmts { + sb.WriteString(tree.SerializeForDisplay(stmt)) + sb.WriteString(";\n") + } + sqlDB.Exec(t, sb.String()) + + // To prevent the test from being flaky, pass the test if PopulateTableWithRandomData + // inserts at least one row in at least one table. + success := false + for i := 1; i <= numTables; i++ { + tableName := tablePrefix + fmt.Sprint(i) + numRows := 30 + numRowsInserted, err := PopulateTableWithRandData(rng, dbConn, tableName, numRows) + require.NoError(t, err) + res := sqlDB.QueryStr(t, fmt.Sprintf("SELECT count(*) FROM %s", tableName)) + require.Equal(t, fmt.Sprint(numRowsInserted), res[0][0]) + if numRowsInserted > 0 { + success = true + break + } + } + require.Equal(t, true, success) +} diff --git a/pkg/ts/server.go b/pkg/ts/server.go index 74763e4090d7..2e76d50f42a3 100644 --- a/pkg/ts/server.go +++ b/pkg/ts/server.go @@ -37,10 +37,10 @@ const ( // queryWorkerMax is the default maximum number of worker goroutines that // the time series server can use to service incoming queries. queryWorkerMax = 8 - // queryMemoryMax is a soft limit for the amount of total memory used by - // time series queries. This is not currently enforced, but is used for - // monitoring purposes. - queryMemoryMax = int64(64 * 1024 * 1024) // 64MiB + // DefaultQueryMemoryMax is a soft limit for the amount of total + // memory used by time series queries. This is not currently enforced, + // but is used for monitoring purposes. + DefaultQueryMemoryMax = int64(64 * 1024 * 1024) // 64MiB // dumpBatchSize is the number of keys processed in each batch by the dump // command. dumpBatchSize = 100 @@ -104,50 +104,54 @@ func MakeServer( db *DB, nodeCountFn ClusterNodeCountFn, cfg ServerConfig, + memoryMonitor *mon.BytesMonitor, stopper *stop.Stopper, ) Server { ambient.AddLogTag("ts-srv", nil) + ctx := ambient.AnnotateCtx(context.Background()) // Override default values from configuration. queryWorkerMax := queryWorkerMax if cfg.QueryWorkerMax != 0 { queryWorkerMax = cfg.QueryWorkerMax } - queryMemoryMax := queryMemoryMax - if cfg.QueryMemoryMax != 0 { + queryMemoryMax := DefaultQueryMemoryMax + if cfg.QueryMemoryMax > DefaultQueryMemoryMax { queryMemoryMax = cfg.QueryMemoryMax } workerSem := quotapool.NewIntPool("ts.Server worker", uint64(queryWorkerMax)) stopper.AddCloser(workerSem.Closer("stopper")) - return Server{ + s := Server{ AmbientContext: ambient, db: db, stopper: stopper, nodeCountFn: nodeCountFn, - workerMemMonitor: mon.NewUnlimitedMonitor( - context.Background(), + workerMemMonitor: mon.NewMonitorInheritWithLimit( "timeseries-workers", - mon.MemoryResource, - nil, - nil, - // Begin logging messages if we exceed our planned memory usage by - // more than double. queryMemoryMax*2, - db.st, + memoryMonitor, ), - resultMemMonitor: mon.NewUnlimitedMonitor( - context.Background(), + resultMemMonitor: mon.NewMonitorInheritWithLimit( "timeseries-results", - mon.MemoryResource, - nil, - nil, math.MaxInt64, - db.st, + memoryMonitor, ), queryMemoryMax: queryMemoryMax, queryWorkerMax: queryWorkerMax, workerSem: workerSem, } + + s.workerMemMonitor.Start(ctx, memoryMonitor, mon.BoundAccount{}) + stopper.AddCloser(stop.CloserFn(func() { + s.workerMemMonitor.Stop(ctx) + })) + + s.resultMemMonitor.Start(ambient.AnnotateCtx(context.Background()), memoryMonitor, mon.BoundAccount{}) + stopper.AddCloser(stop.CloserFn(func() { + s.resultMemMonitor.Stop(ctx) + })) + + return s } // RegisterService registers the GRPC service.