github.com/matrixorigin/matrixone@v1.2.0/pkg/util/export/merge.go (about)

     1  // Copyright 2022 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 export
    16  
    17  import (
    18  	"container/list"
    19  	"context"
    20  	"fmt"
    21  	"io"
    22  	"path"
    23  	"strconv"
    24  	"strings"
    25  	"sync"
    26  	"sync/atomic"
    27  	"time"
    28  
    29  	"github.com/matrixorigin/matrixone/pkg/catalog"
    30  	"github.com/matrixorigin/matrixone/pkg/defines"
    31  
    32  	"github.com/matrixorigin/matrixone/pkg/common/log"
    33  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    34  	"github.com/matrixorigin/matrixone/pkg/common/mpool"
    35  	"github.com/matrixorigin/matrixone/pkg/common/runtime"
    36  	"github.com/matrixorigin/matrixone/pkg/common/util"
    37  	"github.com/matrixorigin/matrixone/pkg/config"
    38  	"github.com/matrixorigin/matrixone/pkg/fileservice"
    39  	"github.com/matrixorigin/matrixone/pkg/logutil"
    40  	"github.com/matrixorigin/matrixone/pkg/pb/task"
    41  	"github.com/matrixorigin/matrixone/pkg/taskservice"
    42  	"github.com/matrixorigin/matrixone/pkg/util/export/etl"
    43  	db_holder "github.com/matrixorigin/matrixone/pkg/util/export/etl/db"
    44  	"github.com/matrixorigin/matrixone/pkg/util/export/table"
    45  	"github.com/matrixorigin/matrixone/pkg/util/trace"
    46  
    47  	"github.com/matrixorigin/simdcsv"
    48  	"go.uber.org/zap"
    49  )
    50  
    51  const LoggerNameETLMerge = "ETLMerge"
    52  const LoggerNameContentReader = "ETLContentReader"
    53  
    54  const MAX_MERGE_INSERT_TIME = 10 * time.Second
    55  
    56  const defaultMaxFileSize = 32 * mpool.MB
    57  
    58  // ========================
    59  // handle merge
    60  // ========================
    61  
    62  // Merge like a compaction, merge input files into one/two/... files.
    63  // - NewMergeService init merge as service, with serviceInited to avoid multi init.
    64  // - MergeTaskExecutorFactory drive by Cron TaskService.
    65  // - NewMerge handle merge obj init.
    66  // - Merge.Start() as service loop, trigger Merge.Main()
    67  // - Merge.Main() handle main job.
    68  //  1. foreach account, build `rootPath` with tuple {account, date, Table }
    69  //  2. call Merge.doMergeFiles() with all files in `rootPath`,  do merge job
    70  //
    71  // - Merge.doMergeFiles handle one job flow: read each file, merge in cache, write into file.
    72  type Merge struct {
    73  	task        task.AsyncTask          // set by WithTask
    74  	table       *table.Table            // set by WithTable
    75  	fs          fileservice.FileService // set by WithFileService
    76  	pathBuilder table.PathBuilder       // const as table.NewAccountDatePathBuilder()
    77  
    78  	// MaxFileSize the total filesize to trigger doMergeFiles(),default: 32 MB
    79  	// Deprecated
    80  	MaxFileSize int64 // set by WithMaxFileSize
    81  	// MaxMergeJobs 允许进行的 Merge 的任务个数,default: 1
    82  	MaxMergeJobs int64 // set by WithMaxMergeJobs
    83  
    84  	// logger
    85  	logger *log.MOLogger
    86  	// mp for TAEReader if needed.
    87  	mp *mpool.MPool
    88  	// runningJobs control task concurrency, init with MaxMergeJobs cnt
    89  	runningJobs chan struct{}
    90  
    91  	// flow ctrl
    92  	ctx        context.Context
    93  	cancelFunc context.CancelFunc
    94  }
    95  
    96  type MergeOption func(*Merge)
    97  
    98  func (opt MergeOption) Apply(m *Merge) {
    99  	opt(m)
   100  }
   101  
   102  func WithTask(task task.AsyncTask) MergeOption {
   103  	return MergeOption(func(m *Merge) {
   104  		m.task = task
   105  	})
   106  }
   107  func WithTable(tbl *table.Table) MergeOption {
   108  	return MergeOption(func(m *Merge) {
   109  		m.table = tbl
   110  	})
   111  }
   112  func WithFileService(fs fileservice.FileService) MergeOption {
   113  	return MergeOption(func(m *Merge) {
   114  		m.fs = fs
   115  	})
   116  }
   117  func WithMaxFileSize(filesize int64) MergeOption {
   118  	return MergeOption(func(m *Merge) {
   119  		m.MaxFileSize = filesize
   120  	})
   121  }
   122  func WithMaxMergeJobs(jobs int64) MergeOption {
   123  	return MergeOption(func(m *Merge) {
   124  		m.MaxMergeJobs = jobs
   125  	})
   126  }
   127  
   128  // serviceInited handle Merge as service
   129  var serviceInited uint32
   130  
   131  func NewMergeService(ctx context.Context, opts ...MergeOption) (*Merge, bool, error) {
   132  	// fix multi-init in standalone
   133  	if !atomic.CompareAndSwapUint32(&serviceInited, 0, 1) {
   134  		return nil, true, nil
   135  	}
   136  	m, err := NewMerge(ctx, opts...)
   137  	return m, false, err
   138  }
   139  
   140  var poolMux sync.Mutex
   141  var ETLMergeTaskPool *mpool.MPool
   142  
   143  func getMpool() (*mpool.MPool, error) {
   144  	poolMux.Lock()
   145  	defer poolMux.Unlock()
   146  	if ETLMergeTaskPool == nil {
   147  		mp, err := mpool.NewMPool("etl_merge_task", 0, mpool.NoFixed)
   148  		if err != nil {
   149  			return nil, err
   150  		}
   151  		ETLMergeTaskPool = mp
   152  	}
   153  	return ETLMergeTaskPool, nil
   154  }
   155  
   156  func NewMerge(ctx context.Context, opts ...MergeOption) (*Merge, error) {
   157  	var err error
   158  	m := &Merge{
   159  		pathBuilder:  table.NewAccountDatePathBuilder(),
   160  		MaxFileSize:  defaultMaxFileSize,
   161  		MaxMergeJobs: 1,
   162  		logger:       runtime.ProcessLevelRuntime().Logger().WithContext(ctx).Named(LoggerNameETLMerge),
   163  	}
   164  	m.ctx, m.cancelFunc = context.WithCancel(ctx)
   165  	for _, opt := range opts {
   166  		opt(m)
   167  	}
   168  	if m.mp, err = getMpool(); err != nil {
   169  		return nil, err
   170  	}
   171  	m.validate(ctx)
   172  	m.runningJobs = make(chan struct{}, m.MaxMergeJobs)
   173  	return m, nil
   174  }
   175  
   176  // validate check missing init elems. Panic with has missing elems.
   177  func (m *Merge) validate(ctx context.Context) {
   178  	if m.table == nil {
   179  		panic(moerr.NewInternalError(ctx, "merge task missing input 'table'"))
   180  	}
   181  	if m.fs == nil {
   182  		panic(moerr.NewInternalError(ctx, "merge task missing input 'FileService'"))
   183  	}
   184  }
   185  
   186  // Start for service Loop
   187  func (m *Merge) Start(ctx context.Context, interval time.Duration) {
   188  	ticker := time.NewTicker(interval)
   189  	defer ticker.Stop()
   190  	for {
   191  		select {
   192  		case <-ticker.C:
   193  			m.Main(ctx)
   194  		case <-m.ctx.Done():
   195  			return
   196  		}
   197  	}
   198  }
   199  
   200  // Stop should call only once
   201  func (m *Merge) Stop() {
   202  	m.cancelFunc()
   203  }
   204  
   205  // =======================
   206  // main logic
   207  // =======================
   208  
   209  type FileMeta struct {
   210  	FilePath string
   211  	FileSize int64
   212  }
   213  
   214  // Main do list all accounts, all dates which belong to m.table.GetName()
   215  func (m *Merge) Main(ctx context.Context) error {
   216  	var files = make([]*FileMeta, 0, 1000)
   217  	var totalSize int64
   218  
   219  	accounts, err := m.fs.List(ctx, "/")
   220  	if err != nil {
   221  		return err
   222  	}
   223  	if len(accounts) == 0 {
   224  		m.logger.Info("merge find empty data")
   225  		return nil
   226  	}
   227  	m.logger.Debug(fmt.Sprintf("merge task with max file: %v MB", m.MaxFileSize/mpool.MB))
   228  	for _, account := range accounts {
   229  		if !account.IsDir {
   230  			m.logger.Warn(fmt.Sprintf("path is not dir: %s", account.Name))
   231  			continue
   232  		}
   233  		// build targetPath like "${account}/logs/*/*/*/${table_name}"
   234  		targetPath := m.pathBuilder.Build(account.Name, table.MergeLogTypeLogs, table.ETLParamTSAll, m.table.GetDatabase(), m.table.GetName())
   235  
   236  		// search all paths like:
   237  		// 0: ${account}/logs/2023/05/31/${table_name}
   238  		// 1: ${account}/logs/2023/06/01/${table_name}
   239  		// 2: ...
   240  		rootPaths, err := m.getAllTargetPath(ctx, targetPath)
   241  		if err != nil {
   242  			return err
   243  		}
   244  
   245  		// get all file entry
   246  		for _, rootPath := range rootPaths {
   247  			m.logger.Info("start merge", logutil.TableField(m.table.GetIdentify()), logutil.PathField(rootPath),
   248  				zap.String("metadata.ID", m.task.Metadata.ID))
   249  
   250  			fileEntrys, err := m.fs.List(ctx, rootPath)
   251  			if err != nil {
   252  				// fixme: m.logger.Error()
   253  				return err
   254  			}
   255  			files = files[:0]
   256  			totalSize = 0
   257  			for _, f := range fileEntrys {
   258  				filepath := path.Join(rootPath, f.Name)
   259  				totalSize += f.Size
   260  				files = append(files, &FileMeta{filepath, f.Size})
   261  				if totalSize > m.MaxFileSize {
   262  					if err = m.doMergeFiles(ctx, files); err != nil {
   263  						m.logger.Error(fmt.Sprintf("merge task meet error: %v", err))
   264  					}
   265  					files = files[:0]
   266  					totalSize = 0
   267  				}
   268  			}
   269  
   270  			if len(files) > 0 {
   271  				if err = m.doMergeFiles(ctx, files); err != nil {
   272  					m.logger.Warn(fmt.Sprintf("merge task meet error: %v", err))
   273  				}
   274  			}
   275  		}
   276  	}
   277  
   278  	return err
   279  }
   280  
   281  func (m *Merge) getAllTargetPath(ctx context.Context, filePath string) ([]string, error) {
   282  	sep := "/"
   283  	pathDir := strings.Split(filePath, sep)
   284  	l := list.New()
   285  	if pathDir[0] == "" {
   286  		l.PushBack(sep)
   287  	} else {
   288  		l.PushBack(pathDir[0])
   289  	}
   290  
   291  	for i := 1; i < len(pathDir); i++ {
   292  		length := l.Len()
   293  		for j := 0; j < length; j++ {
   294  			elem := l.Remove(l.Front())
   295  			prefix := elem.(string)
   296  			entries, err := m.fs.List(ctx, prefix)
   297  			if err != nil {
   298  				return nil, err
   299  			}
   300  			for _, entry := range entries {
   301  				if !entry.IsDir && i+1 != len(pathDir) {
   302  					continue
   303  				}
   304  				matched, err := path.Match(pathDir[i], entry.Name)
   305  				if err != nil {
   306  					return nil, err
   307  				}
   308  				if !matched {
   309  					continue
   310  				}
   311  				l.PushBack(path.Join(prefix, entry.Name))
   312  			}
   313  		}
   314  	}
   315  
   316  	length := l.Len()
   317  	fileList := make([]string, 0, length)
   318  	for idx := 0; idx < length; idx++ {
   319  		fileList = append(fileList, l.Remove(l.Front()).(string))
   320  	}
   321  	return fileList, nil
   322  }
   323  
   324  // doMergeFiles handle merge (read->write->delete) ops for all files in the target directory.
   325  // Handle the files one by one, act uploadFile and do the deletion if upload is success.
   326  // Upload the files to SQL table
   327  // Delete the files from FileService
   328  func (m *Merge) doMergeFiles(ctx context.Context, files []*FileMeta) error {
   329  	ctx, span := trace.Start(ctx, "doMergeFiles")
   330  	defer span.End()
   331  
   332  	// Control task concurrency
   333  	m.runningJobs <- struct{}{}
   334  	defer func() {
   335  		<-m.runningJobs
   336  	}()
   337  
   338  	// Step 3. do simple merge
   339  	var uploadFile = func(ctx context.Context, fp *FileMeta) error {
   340  		row := m.table.GetRow(ctx)
   341  		defer row.Free()
   342  		// open reader
   343  		reader, err := newETLReader(ctx, m.table, m.fs, fp.FilePath, fp.FileSize, m.mp)
   344  		if err != nil {
   345  			m.logger.Error(fmt.Sprintf("merge file meet read failed: %v", err))
   346  			return err
   347  		}
   348  		defer reader.Close()
   349  
   350  		cacheFileData := &SliceCache{}
   351  		defer cacheFileData.Reset()
   352  
   353  		// Read the first line to check if the record already exists
   354  		var existed bool
   355  		firstLine, err := reader.ReadLine()
   356  		if err != nil {
   357  			m.logger.Error("failed to read the first line of the file",
   358  				logutil.PathField(fp.FilePath), zap.Error(err))
   359  			return err
   360  		}
   361  
   362  		if firstLine != nil {
   363  			if err = row.ParseRow(firstLine); err != nil {
   364  				m.logger.Error("parse first ETL row failed",
   365  					logutil.TableField(m.table.GetIdentify()),
   366  					logutil.PathField(fp.FilePath),
   367  					logutil.VarsField(SubStringPrefixLimit(fmt.Sprintf("%v", firstLine), 102400)),
   368  				)
   369  				return err
   370  			}
   371  
   372  			// Check if the first record already exists in the database
   373  			existed, err = db_holder.IsRecordExisted(ctx, firstLine, m.table, db_holder.GetOrInitDBConn)
   374  			if err != nil {
   375  				m.logger.Error("error checking if the first record exists",
   376  					logutil.TableField(m.table.GetIdentify()),
   377  					logutil.PathField(fp.FilePath),
   378  					logutil.ErrorField(err),
   379  				)
   380  				return err
   381  			}
   382  
   383  			// Process the first line since it doesn't exist in the database
   384  			if !existed {
   385  				cacheFileData.Put(row)
   386  			}
   387  		}
   388  
   389  		// read all content if not existed
   390  		if !existed {
   391  			var line []string
   392  			line, err = reader.ReadLine()
   393  			for ; line != nil && err == nil; line, err = reader.ReadLine() {
   394  				if err = row.ParseRow(line); err != nil {
   395  					m.logger.Error("parse ETL rows failed",
   396  						logutil.TableField(m.table.GetIdentify()),
   397  						logutil.PathField(fp.FilePath),
   398  						logutil.VarsField(SubStringPrefixLimit(fmt.Sprintf("%v", line), 102400)),
   399  					)
   400  					return err
   401  				}
   402  
   403  				cacheFileData.Put(row)
   404  			}
   405  			if err != nil {
   406  				m.logger.Warn("failed to read file",
   407  					logutil.PathField(fp.FilePath), zap.Error(err))
   408  				return err
   409  			}
   410  		}
   411  
   412  		// sql insert
   413  		if cacheFileData.Size() > 0 {
   414  			if err = cacheFileData.Flush(m.table); err != nil {
   415  				return err
   416  			}
   417  			cacheFileData.Reset()
   418  		}
   419  		// delete empty file or file already uploaded
   420  		if cacheFileData.Size() == 0 {
   421  			if err = m.fs.Delete(ctx, fp.FilePath); err != nil {
   422  				m.logger.Warn("failed to delete file", zap.Error(err))
   423  				return err
   424  			}
   425  		}
   426  		return nil
   427  	}
   428  	var err error
   429  
   430  	for _, fp := range files {
   431  		if err = uploadFile(ctx, fp); err != nil {
   432  			// todo: adjust the sleep settings
   433  			// Sleep 10 seconds to wait for the database to recover
   434  			time.Sleep(10 * time.Second)
   435  			m.logger.Error("failed to upload file to MO",
   436  				logutil.TableField(m.table.GetIdentify()),
   437  				logutil.PathField(fp.FilePath),
   438  				zap.Error(err),
   439  			)
   440  		}
   441  	}
   442  	logutil.Info("upload files success", logutil.TableField(m.table.GetIdentify()), zap.Int("file count", len(files)))
   443  
   444  	return err
   445  }
   446  
   447  func SubStringPrefixLimit(str string, length int) string {
   448  	if length <= 0 {
   449  		return ""
   450  	}
   451  
   452  	if len(str) < length {
   453  		return str
   454  	} else {
   455  		return str[:length] + "..."
   456  	}
   457  }
   458  
   459  type ContentReader struct {
   460  	ctx     context.Context
   461  	path    string
   462  	idx     int
   463  	length  int
   464  	content [][]string
   465  
   466  	logger *log.MOLogger
   467  	reader *simdcsv.Reader
   468  	raw    io.ReadCloser
   469  }
   470  
   471  // BatchReadRows ~= 20MB rawlog file has about 3700+ rows
   472  const BatchReadRows = 4000
   473  
   474  func NewContentReader(ctx context.Context, path string, reader *simdcsv.Reader, raw io.ReadCloser) *ContentReader {
   475  	logger := runtime.ProcessLevelRuntime().Logger().WithContext(ctx).Named(LoggerNameContentReader)
   476  	return &ContentReader{
   477  		ctx:     ctx,
   478  		path:    path,
   479  		length:  0,
   480  		content: make([][]string, BatchReadRows),
   481  		logger:  logger,
   482  		reader:  reader,
   483  		raw:     raw,
   484  	}
   485  }
   486  
   487  func (s *ContentReader) ReadLine() ([]string, error) {
   488  	if s.idx == s.length && s.reader != nil {
   489  		var cnt int
   490  		var err error
   491  		s.content, cnt, err = s.reader.Read(BatchReadRows, s.ctx, s.content)
   492  		if err != nil {
   493  			return nil, err
   494  		} else if s.content == nil {
   495  			s.logger.Error("ContentReader.ReadLine.nil", logutil.PathField(s.path),
   496  				zap.Bool("nil", s.content == nil),
   497  				zap.Error(s.ctx.Err()),
   498  				zap.Bool("SupportedCPU", simdcsv.SupportedCPU()),
   499  			)
   500  			return nil, moerr.NewInternalError(s.ctx, "read files meet context Done")
   501  		}
   502  		if cnt < BatchReadRows {
   503  			//s.reader.Close() // DO NOT call, because it is a forever loop with empty op.
   504  			s.reader = nil
   505  			s.raw.Close()
   506  			s.raw = nil
   507  			s.logger.Debug("ContentReader.ReadLine.EOF", logutil.PathField(s.path), zap.Int("rows", cnt))
   508  		}
   509  		s.idx = 0
   510  		s.length = cnt
   511  		s.logger.Debug("ContentReader.ReadLine", logutil.PathField(s.path), zap.Int("rows", cnt),
   512  			zap.Bool("SupportedCPU", simdcsv.SupportedCPU()),
   513  		)
   514  	}
   515  	if s.idx < s.length {
   516  		idx := s.idx
   517  		s.idx++
   518  		if s.content == nil || len(s.content) == 0 {
   519  			s.logger.Error("ContentReader.ReadLine.nil",
   520  				logutil.PathField(s.path),
   521  				zap.Bool("nil", s.content == nil),
   522  				zap.Int("cached", len(s.content)),
   523  				zap.Int("idx", idx),
   524  				zap.Bool("SupportedCPU", simdcsv.SupportedCPU()),
   525  			)
   526  		}
   527  		return s.content[idx], nil
   528  	}
   529  	return nil, nil
   530  }
   531  
   532  func (s *ContentReader) ReadRow(row *table.Row) error {
   533  	panic("NOT implement")
   534  }
   535  
   536  func (s *ContentReader) Close() {
   537  	capLen := cap(s.content)
   538  	s.content = s.content[:capLen]
   539  	for idx := range s.content {
   540  		s.content[idx] = nil
   541  	}
   542  	if s.raw != nil {
   543  		_ = s.raw.Close()
   544  		s.raw = nil
   545  	}
   546  }
   547  
   548  func newETLReader(ctx context.Context, tbl *table.Table, fs fileservice.FileService, path string, size int64, mp *mpool.MPool) (ETLReader, error) {
   549  	if strings.LastIndex(path, table.CsvExtension) > 0 {
   550  		return NewCSVReader(ctx, fs, path)
   551  	} else if strings.LastIndex(path, table.TaeExtension) > 0 {
   552  		r, err := etl.NewTaeReader(ctx, tbl, path, size, fs, mp)
   553  		if err != nil {
   554  			r.Close()
   555  			return nil, err
   556  		}
   557  		_, err = r.ReadAll(ctx)
   558  		if err != nil {
   559  			r.Close()
   560  			return nil, err
   561  		}
   562  		return r, nil
   563  	} else {
   564  		panic("NOT Implements")
   565  	}
   566  }
   567  
   568  // NewCSVReader create new csv reader.
   569  // success case return: ok_reader, nil error
   570  // failed case return: nil_reader, error
   571  func NewCSVReader(ctx context.Context, fs fileservice.FileService, path string) (ETLReader, error) {
   572  	// external.ReadFile
   573  	var reader io.ReadCloser
   574  	vec := fileservice.IOVector{
   575  		FilePath: path,
   576  		Entries: []fileservice.IOEntry{
   577  			0: {
   578  				Offset:            0,
   579  				Size:              -1,
   580  				ReadCloserForRead: &reader,
   581  			},
   582  		},
   583  	}
   584  	// open file reader
   585  	if err := fs.Read(ctx, &vec); err != nil {
   586  		return nil, err
   587  	}
   588  
   589  	// parse csv content
   590  	simdCsvReader := simdcsv.NewReaderWithOptions(reader,
   591  		table.CommonCsvOptions.FieldTerminator,
   592  		'#',
   593  		true,
   594  		true)
   595  
   596  	// return content Reader
   597  	return NewContentReader(ctx, path, simdCsvReader, reader), nil
   598  }
   599  
   600  type Cache interface {
   601  	Put(*table.Row)
   602  	Size() int64
   603  	Flush(*table.Table) error
   604  	Reset()
   605  	IsEmpty() bool
   606  }
   607  
   608  type SliceCache struct {
   609  	m    [][]string
   610  	size int64
   611  }
   612  
   613  func (c *SliceCache) Flush(tbl *table.Table) error {
   614  	_, err := db_holder.WriteRowRecords(c.m, tbl, MAX_MERGE_INSERT_TIME)
   615  	c.Reset()
   616  	return err
   617  }
   618  
   619  func (c *SliceCache) Reset() {
   620  	for idx := range c.m {
   621  		c.m[idx] = nil
   622  	}
   623  	c.m = c.m[:0]
   624  	c.size = 0
   625  }
   626  
   627  func (c *SliceCache) IsEmpty() bool {
   628  	return len(c.m) == 0
   629  }
   630  
   631  func (c *SliceCache) Put(r *table.Row) {
   632  	c.m = append(c.m, r.GetCsvStrings())
   633  	c.size += r.Size()
   634  }
   635  
   636  func (c *SliceCache) Size() int64 { return c.size }
   637  
   638  func LongRunETLMerge(ctx context.Context, task task.AsyncTask, logger *log.MOLogger, opts ...MergeOption) error {
   639  	// should init once in/with schema-init.
   640  	tables := table.GetAllTables()
   641  	if len(tables) == 0 {
   642  		logger.Info("empty tables")
   643  		return nil
   644  	}
   645  
   646  	var newOptions []MergeOption
   647  	newOptions = append(newOptions, opts...)
   648  	newOptions = append(newOptions, WithTask(task))
   649  	newOptions = append(newOptions, WithTable(tables[0]))
   650  	merge, err := NewMerge(ctx, newOptions...)
   651  	if err != nil {
   652  		return err
   653  	}
   654  
   655  	logger.Info("start LongRunETLMerge")
   656  	// handle today
   657  	for _, tbl := range tables {
   658  		merge.table = tbl
   659  		if err = merge.Main(ctx); err != nil {
   660  			logger.Error("merge metric failed", zap.Error(err))
   661  		}
   662  	}
   663  
   664  	return nil
   665  }
   666  
   667  func MergeTaskExecutorFactory(opts ...MergeOption) func(ctx context.Context, task task.Task) error {
   668  
   669  	CronMerge := func(ctx context.Context, t task.Task) error {
   670  		asyncTask, ok := t.(*task.AsyncTask)
   671  		if !ok {
   672  			return moerr.NewInternalError(ctx, "invalid task type")
   673  		}
   674  		ctx, span := trace.Start(ctx, "CronMerge")
   675  		defer span.End()
   676  
   677  		args := asyncTask.Metadata.Context
   678  		ts := time.Now()
   679  		logger := runtime.ProcessLevelRuntime().Logger().WithContext(ctx).Named(LoggerNameETLMerge)
   680  		fields := []zap.Field{
   681  			zap.String("args", util.UnsafeBytesToString(args)),
   682  			zap.Time("start", ts),
   683  			zap.Uint64("taskID", asyncTask.ID),
   684  			zap.Int64("create", asyncTask.CreateAt),
   685  			zap.String("metadataID", asyncTask.Metadata.ID),
   686  		}
   687  		logger.Info("start merge", fields...)
   688  		defer logger.Info("done merge", fields...)
   689  
   690  		// task run long time
   691  		if len(args) != 0 {
   692  			logger.Warn("ETLMergeTask should have empty args", zap.Int("cnt", len(args)))
   693  		}
   694  		if err := LongRunETLMerge(ctx, *asyncTask, logger, opts...); err != nil {
   695  			return err
   696  		}
   697  		return nil
   698  	}
   699  	return CronMerge
   700  }
   701  
   702  // MergeTaskCronExpr support sec level
   703  // Deprecated
   704  var MergeTaskCronExpr = MergeTaskCronExprEvery4Hour
   705  
   706  const MergeTaskCronExprEvery15Sec = "*/15 * * * * *"
   707  const MergeTaskCronExprEveryMin = "0 * * * * *"
   708  const MergeTaskCronExprEvery05Min = "0 */5 * * * *"
   709  const MergeTaskCronExprEvery15Min = "0 */15 * * * *"
   710  const MergeTaskCronExprEvery1Hour = "0 0 */1 * * *"
   711  const MergeTaskCronExprEvery2Hour = "0 0 */2 * * *"
   712  const MergeTaskCronExprEvery4Hour = "0 0 4,8,12,16,20 * * *"
   713  const MergeTaskCronExprYesterday = "0 5 0 * * *"
   714  const MergeTaskToday = "today"
   715  const MergeTaskYesterday = "yesterday"
   716  const ParamSeparator = " "
   717  
   718  // MergeTaskMetadata handle args like: "{db_tbl_name} [date, default: today]"
   719  func MergeTaskMetadata(id task.TaskCode, args ...string) task.TaskMetadata {
   720  	return task.TaskMetadata{
   721  		ID:       path.Join("ETLMergeTask", path.Join(args...)),
   722  		Executor: id,
   723  		Context:  []byte(strings.Join(args, ParamSeparator)),
   724  		Options:  task.TaskOptions{Concurrency: 1},
   725  	}
   726  }
   727  
   728  func CreateCronTask(ctx context.Context, executorID task.TaskCode, taskService taskservice.TaskService) error {
   729  	var err error
   730  	ctx, span := trace.Start(ctx, "ETLMerge.CreateCronTask")
   731  	defer span.End()
   732  	ctx = defines.AttachAccount(ctx, catalog.System_Account, catalog.System_User, catalog.System_Role)
   733  	logger := runtime.ProcessLevelRuntime().Logger().WithContext(ctx)
   734  	logger.Info(fmt.Sprintf("init merge task with CronExpr: %s", MergeTaskCronExpr))
   735  	if err = taskService.CreateCronTask(ctx, MergeTaskMetadata(executorID), MergeTaskCronExpr); err != nil {
   736  		return err
   737  	}
   738  	return nil
   739  }
   740  
   741  // InitCronExpr support min interval 5 min, max 12 hour
   742  func InitCronExpr(ctx context.Context, duration time.Duration) error {
   743  	if duration < 0 || duration > 12*time.Hour {
   744  		return moerr.NewNotSupported(ctx, "export cron expr not support cycle: %v", duration)
   745  	}
   746  	if duration < 5*time.Minute {
   747  		MergeTaskCronExpr = fmt.Sprintf("@every %.0fs", duration.Seconds())
   748  	} else if duration < time.Hour {
   749  		const unit = 5 * time.Minute
   750  		duration = (duration + unit - 1) / unit * unit
   751  		switch duration {
   752  		case 5 * time.Minute:
   753  			MergeTaskCronExpr = MergeTaskCronExprEvery05Min
   754  		case 15 * time.Minute:
   755  			MergeTaskCronExpr = MergeTaskCronExprEvery15Min
   756  		default:
   757  			MergeTaskCronExpr = fmt.Sprintf("@every %.0fm", duration.Minutes())
   758  		}
   759  	} else {
   760  		minHour := duration / time.Hour
   761  		switch minHour {
   762  		case 1:
   763  			MergeTaskCronExpr = MergeTaskCronExprEvery1Hour
   764  		case 2:
   765  			MergeTaskCronExpr = MergeTaskCronExprEvery2Hour
   766  		case 4:
   767  			MergeTaskCronExpr = MergeTaskCronExprEvery4Hour
   768  		default:
   769  			var hours = make([]string, 0, 12)
   770  			for h := minHour; h < 24; h += minHour {
   771  				hours = append(hours, strconv.Itoa(int(h)))
   772  			}
   773  			MergeTaskCronExpr = fmt.Sprintf("0 0 %s * * *", strings.Join(hours, ","))
   774  		}
   775  	}
   776  	return nil
   777  }
   778  
   779  func InitMerge(ctx context.Context, SV *config.ObservabilityParameters) error {
   780  	var err error
   781  	mergeCycle := SV.MergeCycle.Duration
   782  	if mergeCycle > 0 {
   783  		err = InitCronExpr(ctx, mergeCycle)
   784  		if err != nil {
   785  			return err
   786  		}
   787  	}
   788  	return nil
   789  }