github.com/matrixorigin/matrixone@v1.2.0/pkg/sql/compile/sql_executor.go (about)

     1  // Copyright 2023 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package compile
    16  
    17  import (
    18  	"context"
    19  	"encoding/hex"
    20  	"errors"
    21  	"time"
    22  
    23  	"github.com/matrixorigin/matrixone/pkg/catalog"
    24  
    25  	"github.com/matrixorigin/matrixone/pkg/common/buffer"
    26  	"github.com/matrixorigin/matrixone/pkg/common/mpool"
    27  	"github.com/matrixorigin/matrixone/pkg/common/runtime"
    28  	"github.com/matrixorigin/matrixone/pkg/container/batch"
    29  	"github.com/matrixorigin/matrixone/pkg/defines"
    30  	"github.com/matrixorigin/matrixone/pkg/fileservice"
    31  	"github.com/matrixorigin/matrixone/pkg/lockservice"
    32  	"github.com/matrixorigin/matrixone/pkg/logservice"
    33  	"github.com/matrixorigin/matrixone/pkg/logutil"
    34  	qclient "github.com/matrixorigin/matrixone/pkg/queryservice/client"
    35  	"github.com/matrixorigin/matrixone/pkg/sql/parsers"
    36  	"github.com/matrixorigin/matrixone/pkg/sql/parsers/dialect"
    37  	"github.com/matrixorigin/matrixone/pkg/sql/plan"
    38  	"github.com/matrixorigin/matrixone/pkg/txn/client"
    39  	"github.com/matrixorigin/matrixone/pkg/udf"
    40  	"github.com/matrixorigin/matrixone/pkg/util"
    41  	"github.com/matrixorigin/matrixone/pkg/util/executor"
    42  	"github.com/matrixorigin/matrixone/pkg/vm/engine"
    43  	"github.com/matrixorigin/matrixone/pkg/vm/process"
    44  
    45  	"go.uber.org/zap"
    46  )
    47  
    48  type sqlExecutor struct {
    49  	addr      string
    50  	eng       engine.Engine
    51  	mp        *mpool.MPool
    52  	txnClient client.TxnClient
    53  	fs        fileservice.FileService
    54  	ls        lockservice.LockService
    55  	qc        qclient.QueryClient
    56  	hakeeper  logservice.CNHAKeeperClient
    57  	us        udf.Service
    58  	aicm      *defines.AutoIncrCacheManager
    59  	buf       *buffer.Buffer
    60  }
    61  
    62  // NewSQLExecutor returns a internal used sql service. It can execute sql in current CN.
    63  func NewSQLExecutor(
    64  	addr string,
    65  	eng engine.Engine,
    66  	mp *mpool.MPool,
    67  	txnClient client.TxnClient,
    68  	fs fileservice.FileService,
    69  	qc qclient.QueryClient,
    70  	hakeeper logservice.CNHAKeeperClient,
    71  	us udf.Service,
    72  	aicm *defines.AutoIncrCacheManager) executor.SQLExecutor {
    73  	v, ok := runtime.ProcessLevelRuntime().GetGlobalVariables(runtime.LockService)
    74  	if !ok {
    75  		panic("missing lock service")
    76  	}
    77  	return &sqlExecutor{
    78  		addr:      addr,
    79  		eng:       eng,
    80  		txnClient: txnClient,
    81  		fs:        fs,
    82  		ls:        v.(lockservice.LockService),
    83  		qc:        qc,
    84  		hakeeper:  hakeeper,
    85  		us:        us,
    86  		aicm:      aicm,
    87  		mp:        mp,
    88  		buf:       buffer.New(),
    89  	}
    90  }
    91  
    92  func (s *sqlExecutor) NewTxnOperator(ctx context.Context) client.TxnOperator {
    93  	var opts executor.Options
    94  
    95  	ctx, opts, err := s.adjustOptions(ctx, opts)
    96  	if err != nil {
    97  		return nil
    98  	}
    99  	if !opts.ExistsTxn() {
   100  		if err := s.eng.New(ctx, opts.Txn()); err != nil {
   101  			return nil
   102  		}
   103  	}
   104  	opts.Txn().GetWorkspace().StartStatement()
   105  	opts.Txn().GetWorkspace().IncrStatementID(ctx, false)
   106  	return opts.Txn()
   107  }
   108  
   109  func (s *sqlExecutor) Exec(
   110  	ctx context.Context,
   111  	sql string,
   112  	opts executor.Options) (executor.Result, error) {
   113  	var res executor.Result
   114  	err := s.ExecTxn(
   115  		ctx,
   116  		func(exec executor.TxnExecutor) error {
   117  			v, err := exec.Exec(sql, opts.StatementOption())
   118  			res = v
   119  			return err
   120  		},
   121  		opts)
   122  	if err != nil {
   123  		return executor.Result{}, err
   124  	}
   125  	return res, nil
   126  }
   127  
   128  func (s *sqlExecutor) ExecTxn(
   129  	ctx context.Context,
   130  	execFunc func(executor.TxnExecutor) error,
   131  	opts executor.Options) error {
   132  	exec, err := newTxnExecutor(ctx, s, opts)
   133  	if err != nil {
   134  		return err
   135  	}
   136  	err = execFunc(exec)
   137  	if err != nil {
   138  		logutil.Errorf("internal sql executor error: %v", err)
   139  		return exec.rollback(err)
   140  	}
   141  	if err = exec.commit(); err != nil {
   142  		return err
   143  	}
   144  	s.maybeWaitCommittedLogApplied(exec.opts)
   145  	return nil
   146  }
   147  
   148  func (s *sqlExecutor) maybeWaitCommittedLogApplied(opts executor.Options) {
   149  	if !opts.WaitCommittedLogApplied() {
   150  		return
   151  	}
   152  	ts := opts.Txn().Txn().CommitTS
   153  	if !ts.IsEmpty() {
   154  		s.txnClient.SyncLatestCommitTS(ts)
   155  	}
   156  }
   157  
   158  func (s *sqlExecutor) getCompileContext(
   159  	ctx context.Context,
   160  	proc *process.Process,
   161  	db string) *compilerContext {
   162  	return newCompilerContext(
   163  		ctx,
   164  		db,
   165  		s.eng,
   166  		proc)
   167  }
   168  
   169  func (s *sqlExecutor) adjustOptions(
   170  	ctx context.Context,
   171  	opts executor.Options) (context.Context, executor.Options, error) {
   172  	if opts.HasAccountID() {
   173  		ctx = context.WithValue(
   174  			ctx,
   175  			defines.TenantIDKey{},
   176  			opts.AccountID())
   177  	} else if ctx.Value(defines.TenantIDKey{}) == nil {
   178  		ctx = context.WithValue(
   179  			ctx,
   180  			defines.TenantIDKey{},
   181  			uint32(0))
   182  	}
   183  
   184  	if !opts.HasExistsTxn() {
   185  		txnOpts := opts.ExtraTxnOptions()
   186  		txnOpts = append(txnOpts,
   187  			client.WithTxnCreateBy(
   188  				opts.AccountID(),
   189  				"",
   190  				"sql-executor",
   191  				0),
   192  			client.WithDisableTrace(!opts.EnableTrace()))
   193  		txnOp, err := s.txnClient.New(
   194  			ctx,
   195  			opts.MinCommittedTS(),
   196  			txnOpts...)
   197  		if err != nil {
   198  			return nil, executor.Options{}, err
   199  		}
   200  		opts = opts.SetupNewTxn(txnOp)
   201  	}
   202  	return ctx, opts, nil
   203  }
   204  
   205  type txnExecutor struct {
   206  	s        *sqlExecutor
   207  	ctx      context.Context
   208  	opts     executor.Options
   209  	database string
   210  }
   211  
   212  func newTxnExecutor(
   213  	ctx context.Context,
   214  	s *sqlExecutor,
   215  	opts executor.Options) (*txnExecutor, error) {
   216  	ctx, opts, err := s.adjustOptions(ctx, opts)
   217  	if err != nil {
   218  		return nil, err
   219  	}
   220  	if !opts.ExistsTxn() {
   221  		if err := s.eng.New(ctx, opts.Txn()); err != nil {
   222  			return nil, err
   223  		}
   224  	}
   225  	return &txnExecutor{s: s, ctx: ctx, opts: opts}, nil
   226  }
   227  
   228  func (exec *txnExecutor) Use(db string) {
   229  	exec.database = db
   230  }
   231  
   232  func (exec *txnExecutor) Exec(
   233  	sql string,
   234  	statementOption executor.StatementOption) (executor.Result, error) {
   235  
   236  	//-----------------------------------------------------------------------------------------
   237  	// NOTE: This code is to restore tenantID information in the Context when temporarily switching tenants
   238  	// so that it can be restored to its original state after completing the task.
   239  	recoverAccount := func(exec *txnExecutor, accId uint32) {
   240  		exec.ctx = context.WithValue(exec.ctx, defines.TenantIDKey{}, accId)
   241  	}
   242  
   243  	if statementOption.HasAccountID() {
   244  		originAccountID := catalog.System_Account
   245  		if v := exec.ctx.Value(defines.TenantIDKey{}); v != nil {
   246  			originAccountID = v.(uint32)
   247  		}
   248  
   249  		exec.ctx = context.WithValue(exec.ctx,
   250  			defines.TenantIDKey{},
   251  			statementOption.AccountID())
   252  		// NOTE: Restore AccountID information in context.Context
   253  		defer recoverAccount(exec, originAccountID)
   254  	}
   255  	//-----------------------------------------------------------------------------------------
   256  
   257  	receiveAt := time.Now()
   258  
   259  	stmts, err := parsers.Parse(exec.ctx, dialect.MYSQL, sql, 1, 0)
   260  	defer func() {
   261  		for _, stmt := range stmts {
   262  			stmt.Free()
   263  		}
   264  	}()
   265  	if err != nil {
   266  		return executor.Result{}, err
   267  	}
   268  
   269  	// TODO(volgariver6): we got a duplicate code logic in `func (cwft *TxnComputationWrapper) Compile`,
   270  	// maybe we should fix it.
   271  	txnOp := exec.opts.Txn()
   272  	if txnOp != nil && !exec.opts.DisableIncrStatement() {
   273  		txnOp.GetWorkspace().StartStatement()
   274  		defer func() {
   275  			txnOp.GetWorkspace().EndStatement()
   276  		}()
   277  
   278  		err := txnOp.GetWorkspace().IncrStatementID(exec.ctx, false)
   279  		if err != nil {
   280  			return executor.Result{}, err
   281  		}
   282  	}
   283  
   284  	proc := process.New(
   285  		exec.ctx,
   286  		exec.s.mp,
   287  		exec.s.txnClient,
   288  		exec.opts.Txn(),
   289  		exec.s.fs,
   290  		exec.s.ls,
   291  		exec.s.qc,
   292  		exec.s.hakeeper,
   293  		exec.s.us,
   294  		exec.s.aicm,
   295  	)
   296  	proc.WaitPolicy = statementOption.WaitPolicy()
   297  	proc.SetVectorPoolSize(0)
   298  	proc.SessionInfo.TimeZone = exec.opts.GetTimeZone()
   299  	proc.SessionInfo.Buf = exec.s.buf
   300  	proc.SessionInfo.StorageEngine = exec.s.eng
   301  	defer func() {
   302  		proc.CleanValueScanBatchs()
   303  		proc.FreeVectors()
   304  	}()
   305  
   306  	compileContext := exec.s.getCompileContext(exec.ctx, proc, exec.getDatabase())
   307  	compileContext.SetRootSql(sql)
   308  
   309  	pn, err := plan.BuildPlan(compileContext, stmts[0], false)
   310  	if err != nil {
   311  		return executor.Result{}, err
   312  	}
   313  
   314  	c := NewCompile(exec.s.addr, exec.getDatabase(), sql, "", "", exec.ctx, exec.s.eng, proc, stmts[0], false, nil, receiveAt)
   315  	defer c.Release()
   316  	c.disableRetry = exec.opts.DisableIncrStatement()
   317  	c.SetBuildPlanFunc(func() (*plan.Plan, error) {
   318  		return plan.BuildPlan(
   319  			exec.s.getCompileContext(exec.ctx, proc, exec.getDatabase()),
   320  			stmts[0], false)
   321  	})
   322  
   323  	result := executor.NewResult(exec.s.mp)
   324  	var batches []*batch.Batch
   325  	err = c.Compile(
   326  		exec.ctx,
   327  		pn,
   328  		func(bat *batch.Batch) error {
   329  			if bat != nil {
   330  				// the bat is valid only in current method. So we need copy data.
   331  				// FIXME: add a custom streaming apply handler to consume readed data. Now
   332  				// our current internal sql will never read too much data.
   333  				rows, err := bat.Dup(exec.s.mp)
   334  				if err != nil {
   335  					return err
   336  				}
   337  				batches = append(batches, rows)
   338  			}
   339  			return nil
   340  		})
   341  	if err != nil {
   342  		return executor.Result{}, err
   343  	}
   344  	var runResult *util.RunResult
   345  	runResult, err = c.Run(0)
   346  	if err != nil {
   347  		for _, bat := range batches {
   348  			if bat != nil {
   349  				bat.Clean(exec.s.mp)
   350  			}
   351  		}
   352  		return executor.Result{}, err
   353  	}
   354  
   355  	logutil.Info("sql_executor exec",
   356  		zap.String("sql", sql),
   357  		zap.String("txn-id", hex.EncodeToString(exec.opts.Txn().Txn().ID)),
   358  		zap.Duration("duration", time.Since(receiveAt)),
   359  		zap.Uint64("AffectedRows", runResult.AffectRows),
   360  	)
   361  	result.LastInsertID = proc.GetLastInsertID()
   362  	result.Batches = batches
   363  	result.AffectedRows = runResult.AffectRows
   364  	return result, nil
   365  }
   366  
   367  func (exec *txnExecutor) LockTable(table string) error {
   368  	txnOp := exec.opts.Txn()
   369  	ctx := exec.ctx
   370  
   371  	dbSource, err := exec.s.eng.Database(ctx, exec.opts.Database(), txnOp)
   372  	if err != nil {
   373  		return err
   374  	}
   375  	rel, err := dbSource.Relation(ctx, table, nil)
   376  	if err != nil {
   377  		return err
   378  	}
   379  	proc := process.New(
   380  		ctx,
   381  		exec.s.mp,
   382  		exec.s.txnClient,
   383  		txnOp,
   384  		exec.s.fs,
   385  		exec.s.ls,
   386  		exec.s.qc,
   387  		exec.s.hakeeper,
   388  		exec.s.us,
   389  		exec.s.aicm,
   390  	)
   391  	proc.SetVectorPoolSize(0)
   392  	proc.SessionInfo.TimeZone = exec.opts.GetTimeZone()
   393  	proc.SessionInfo.Buf = exec.s.buf
   394  	defer func() {
   395  		proc.CleanValueScanBatchs()
   396  		proc.FreeVectors()
   397  	}()
   398  	return doLockTable(exec.s.eng, proc, rel, false)
   399  }
   400  
   401  func (exec *txnExecutor) Txn() client.TxnOperator {
   402  	return exec.opts.Txn()
   403  }
   404  
   405  func (exec *txnExecutor) commit() error {
   406  	if exec.opts.ExistsTxn() {
   407  		return nil
   408  	}
   409  	return exec.opts.Txn().Commit(exec.ctx)
   410  }
   411  
   412  func (exec *txnExecutor) rollback(err error) error {
   413  	if exec.opts.ExistsTxn() {
   414  		return err
   415  	}
   416  	return errors.Join(err, exec.opts.Txn().Rollback(exec.ctx))
   417  }
   418  
   419  func (exec *txnExecutor) getDatabase() string {
   420  	if exec.database != "" {
   421  		return exec.database
   422  	}
   423  	return exec.opts.Database()
   424  }