diff --git a/pkg/bootstrap/service_test.go b/pkg/bootstrap/service_test.go index 01befdf8fd6ef..85088cd633266 100644 --- a/pkg/bootstrap/service_test.go +++ b/pkg/bootstrap/service_test.go @@ -177,7 +177,7 @@ func (tTxnOp *testTxnOperator) GetWorkspace() client.Workspace { panic("implement me") } -func (tTxnOp *testTxnOperator) AppendEventCallback(event client.EventType, callbacks ...func(client.TxnEvent)) { +func (tTxnOp *testTxnOperator) AppendEventCallback(event client.EventType, callbacks ...client.TxnEventCallback) { //TODO implement me panic("implement me") } diff --git a/pkg/frontend/test/engine_mock.go b/pkg/frontend/test/engine_mock.go index f9f1c9030e52d..45c1b2fbe15ee 100644 --- a/pkg/frontend/test/engine_mock.go +++ b/pkg/frontend/test/engine_mock.go @@ -1075,16 +1075,16 @@ func (mr *MockRelationMockRecorder) BuildShardingReaders(ctx, proc, expr, relDat // CollectChanges mocks base method. func (m *MockRelation) CollectChanges(ctx context.Context, from, to types.TS, skipDeletes bool, mp *mpool.MPool) (engine.ChangesHandle, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "CollectChanges", ctx, from, to, mp) + ret := m.ctrl.Call(m, "CollectChanges", ctx, from, to, skipDeletes, mp) ret0, _ := ret[0].(engine.ChangesHandle) ret1, _ := ret[1].(error) return ret0, ret1 } // CollectChanges indicates an expected call of CollectChanges. -func (mr *MockRelationMockRecorder) CollectChanges(ctx, from, to, mp interface{}) *gomock.Call { +func (mr *MockRelationMockRecorder) CollectChanges(ctx, from, to, skipDeletes, mp interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CollectChanges", reflect.TypeOf((*MockRelation)(nil).CollectChanges), ctx, from, to, mp) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CollectChanges", reflect.TypeOf((*MockRelation)(nil).CollectChanges), ctx, from, to, skipDeletes, mp) } // CollectTombstones mocks base method. @@ -1201,6 +1201,7 @@ func (mr *MockRelationMockRecorder) GetExtraInfo() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetExtraInfo", reflect.TypeOf((*MockRelation)(nil).GetExtraInfo)) } +// GetNonAppendableObjectStats mocks base method. func (m *MockRelation) GetNonAppendableObjectStats(ctx context.Context) ([]objectio.ObjectStats, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetNonAppendableObjectStats", ctx) @@ -1318,18 +1319,18 @@ func (mr *MockRelationMockRecorder) MergeObjects(ctx, objstats, targetObjSize in } // PrimaryKeysMayBeModified mocks base method. -func (m *MockRelation) PrimaryKeysMayBeModified(ctx context.Context, from, to types.TS, batch *batch.Batch, pkIndex, _ int32) (bool, error) { +func (m *MockRelation) PrimaryKeysMayBeModified(ctx context.Context, from, to types.TS, batch *batch.Batch, pkIndex, partitionIndex int32) (bool, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "PrimaryKeysMayBeModified", ctx, from, to, batch, pkIndex) + ret := m.ctrl.Call(m, "PrimaryKeysMayBeModified", ctx, from, to, batch, pkIndex, partitionIndex) ret0, _ := ret[0].(bool) ret1, _ := ret[1].(error) return ret0, ret1 } // PrimaryKeysMayBeModified indicates an expected call of PrimaryKeysMayBeModified. -func (mr *MockRelationMockRecorder) PrimaryKeysMayBeModified(ctx, from, to, batch, pkIndex interface{}) *gomock.Call { +func (mr *MockRelationMockRecorder) PrimaryKeysMayBeModified(ctx, from, to, batch, pkIndex, partitionIndex interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PrimaryKeysMayBeModified", reflect.TypeOf((*MockRelation)(nil).PrimaryKeysMayBeModified), ctx, from, to, batch, pkIndex) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PrimaryKeysMayBeModified", reflect.TypeOf((*MockRelation)(nil).PrimaryKeysMayBeModified), ctx, from, to, batch, pkIndex, partitionIndex) } // PrimaryKeysMayBeUpserted mocks base method. @@ -2011,6 +2012,20 @@ func (mr *MockEngineMockRecorder) GetService() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetService", reflect.TypeOf((*MockEngine)(nil).GetService)) } +// HasTempEngine mocks base method. +func (m *MockEngine) HasTempEngine() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "HasTempEngine") + ret0, _ := ret[0].(bool) + return ret0 +} + +// HasTempEngine indicates an expected call of HasTempEngine. +func (mr *MockEngineMockRecorder) HasTempEngine() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HasTempEngine", reflect.TypeOf((*MockEngine)(nil).HasTempEngine)) +} + // Hints mocks base method. func (m *MockEngine) Hints() engine.Hints { m.ctrl.T.Helper() @@ -2039,19 +2054,6 @@ func (mr *MockEngineMockRecorder) LatestLogtailAppliedTime() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LatestLogtailAppliedTime", reflect.TypeOf((*MockEngine)(nil).LatestLogtailAppliedTime)) } -// HasTempEngine mocks base method. -func (m *MockEngine) HasTempEngine() bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "HasTempEngine") - ret0, _ := ret[0].(bool) - return ret0 -} - -func (mr *MockEngineMockRecorder) HasTempEngine() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HasTempEngine", reflect.TypeOf((*MockEngine)(nil).HasTempEngine)) -} - // New mocks base method. func (m *MockEngine) New(ctx context.Context, op client.TxnOperator) error { m.ctrl.T.Helper() diff --git a/pkg/frontend/test/txn_mock.go b/pkg/frontend/test/txn_mock.go index 3da8a2c205d07..8d8dcd116431b 100644 --- a/pkg/frontend/test/txn_mock.go +++ b/pkg/frontend/test/txn_mock.go @@ -459,7 +459,7 @@ func (mr *MockTxnOperatorMockRecorder) AddWorkspace(workspace interface{}) *gomo } // AppendEventCallback mocks base method. -func (m *MockTxnOperator) AppendEventCallback(event client.EventType, callbacks ...func(client.TxnEvent)) { +func (m *MockTxnOperator) AppendEventCallback(event client.EventType, callbacks ...client.TxnEventCallback) { m.ctrl.T.Helper() varargs := []interface{}{event} for _, a := range callbacks { diff --git a/pkg/frontend/txn_test.go b/pkg/frontend/txn_test.go index 7ac6b81d26593..d1422eadb2762 100644 --- a/pkg/frontend/txn_test.go +++ b/pkg/frontend/txn_test.go @@ -931,7 +931,7 @@ func (txnop *testTxnOp) GetWorkspace() client.Workspace { return txnop.wp } -func (txnop *testTxnOp) AppendEventCallback(event client.EventType, callbacks ...func(client.TxnEvent)) { +func (txnop *testTxnOp) AppendEventCallback(event client.EventType, callbacks ...client.TxnEventCallback) { //TODO implement me panic("implement me") } diff --git a/pkg/incrservice/service.go b/pkg/incrservice/service.go index 07fe7d9204d03..5bddcfd6344d4 100644 --- a/pkg/incrservice/service.go +++ b/pkg/incrservice/service.go @@ -96,7 +96,7 @@ func (s *service) Create( txnOp.AppendEventCallback( client.ClosedEvent, - s.txnClosed) + client.NewTxnEventCallback(s.txnClosed)) if err := s.store.Create(ctx, tableID, cols, txnOp); err != nil { s.logger.Error("create auto increment cache failed", zap.Uint64("table-id", tableID), @@ -180,7 +180,7 @@ func (s *service) Delete( txnOp.AppendEventCallback( client.ClosedEvent, - s.txnClosed) + client.NewTxnEventCallback(s.txnClosed)) s.mu.Lock() defer s.mu.Unlock() @@ -345,12 +345,13 @@ func (s *service) getCommittedTableCache( return c, nil } -func (s *service) txnClosed(event client.TxnEvent) { +func (s *service) txnClosed(ctx context.Context, txnOp client.TxnOperator, event client.TxnEvent, v any) error { s.mu.Lock() defer s.mu.Unlock() s.handleCreatesLocked(event.Txn) s.handleDeletesLocked(event.Txn) + return nil } func (s *service) handleCreatesLocked(txnMeta txn.TxnMeta) { diff --git a/pkg/incrservice/store_mem.go b/pkg/incrservice/store_mem.go index f96592743241d..19a2ab1d2c681 100644 --- a/pkg/incrservice/store_mem.go +++ b/pkg/incrservice/store_mem.go @@ -60,17 +60,19 @@ func (s *memStore) Create( s.uncommitted[string(txnOp.Txn().ID)] = m txnOp.AppendEventCallback( client.ClosedEvent, - func(event client.TxnEvent) { - txnMeta := event.Txn - s.Lock() - defer s.Unlock() - delete(s.uncommitted, string(txnMeta.ID)) - if txnMeta.Status == txn.TxnStatus_Committed { - for k, v := range m { - s.caches[k] = v + client.NewTxnEventCallback( + func(ctx context.Context, txnOp client.TxnOperator, event client.TxnEvent, v any) error { + txnMeta := event.Txn + s.Lock() + defer s.Unlock() + delete(s.uncommitted, string(txnMeta.ID)) + if txnMeta.Status == txn.TxnStatus_Committed { + for k, v := range m { + s.caches[k] = v + } } - } - }) + return nil + })) } caches := m[tableID] diff --git a/pkg/incrservice/store_sql_test.go b/pkg/incrservice/store_sql_test.go index 0bc703033699c..f60f7ba5e08f0 100644 --- a/pkg/incrservice/store_sql_test.go +++ b/pkg/incrservice/store_sql_test.go @@ -285,7 +285,7 @@ func (tTxnOp *testTxnOperator) GetWorkspace() client.Workspace { panic("implement me") } -func (tTxnOp *testTxnOperator) AppendEventCallback(event client.EventType, callbacks ...func(client.TxnEvent)) { +func (tTxnOp *testTxnOperator) AppendEventCallback(event client.EventType, callbacks ...client.TxnEventCallback) { //TODO implement me panic("implement me") } diff --git a/pkg/iscp/consumer.go b/pkg/iscp/consumer.go index 62e3e42808a34..bdf1a3eeb8a52 100644 --- a/pkg/iscp/consumer.go +++ b/pkg/iscp/consumer.go @@ -14,20 +14,26 @@ package iscp -import "github.com/matrixorigin/matrixone/pkg/sql/plan" +import ( + "github.com/matrixorigin/matrixone/pkg/sql/plan" + "github.com/matrixorigin/matrixone/pkg/txn/client" + "github.com/matrixorigin/matrixone/pkg/vm/engine" +) func NewConsumer( cnUUID string, + cnEngine engine.Engine, + cnTxnClient client.TxnClient, tableDef *plan.TableDef, jobID JobID, info *ConsumerInfo, ) (Consumer, error) { if info.ConsumerType == int8(ConsumerType_CNConsumer) { - return NewInteralSqlConsumer(cnUUID, tableDef, jobID, info) + return NewInteralSqlConsumer(cnUUID, cnEngine, cnTxnClient, tableDef, jobID, info) } if info.ConsumerType == int8(ConsumerType_IndexSync) { - return NewIndexConsumer(cnUUID, tableDef, jobID, info) + return NewIndexConsumer(cnUUID, cnEngine, cnTxnClient, tableDef, jobID, info) } panic("todo") diff --git a/pkg/iscp/data_retriever.go b/pkg/iscp/data_retriever.go index b8f89d72eacd6..b05cd911ee10a 100644 --- a/pkg/iscp/data_retriever.go +++ b/pkg/iscp/data_retriever.go @@ -18,10 +18,13 @@ import ( "context" "encoding/json" "sync" + "time" + "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/cdc" "github.com/matrixorigin/matrixone/pkg/container/types" - "github.com/matrixorigin/matrixone/pkg/util/executor" + "github.com/matrixorigin/matrixone/pkg/defines" + "github.com/matrixorigin/matrixone/pkg/txn/client" ) func MarshalJobStatus(status *JobStatus) (string, error) { @@ -135,7 +138,14 @@ func (r *DataRetrieverImpl) Next() *ISCPData { return data } -func (r *DataRetrieverImpl) UpdateWatermark(exec executor.TxnExecutor, opts executor.StatementOption) error { +func (r *DataRetrieverImpl) UpdateWatermark(ctx context.Context, + cnUUID string, + txn client.TxnOperator) error { + + ctxWithSysAccount := context.WithValue(ctx, defines.TenantIDKey{}, catalog.System_Account) + ctxWithSysAccount, cancel := context.WithTimeout(ctxWithSysAccount, time.Minute*5) + defer cancel() + if r.typ == ISCPDataType_Snapshot { return nil } @@ -153,8 +163,12 @@ func (r *DataRetrieverImpl) UpdateWatermark(exec executor.TxnExecutor, opts exec statusJson, ISCPJobState_Completed, ) - _, err = exec.Exec(updateWatermarkSQL, opts) - return err + res, err := ExecWithResult(ctxWithSysAccount, updateWatermarkSQL, cnUUID, txn) + if err != nil { + return err + } + defer res.Close() + return nil } func (r *DataRetrieverImpl) GetDataType() int8 { diff --git a/pkg/iscp/index_consumer.go b/pkg/iscp/index_consumer.go index ecdbead009f69..b870b4a6ba7ed 100644 --- a/pkg/iscp/index_consumer.go +++ b/pkg/iscp/index_consumer.go @@ -21,13 +21,18 @@ import ( "sync" "time" + "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" - "github.com/matrixorigin/matrixone/pkg/common/runtime" + "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/txn/client" "github.com/matrixorigin/matrixone/pkg/util/executor" "github.com/matrixorigin/matrixone/pkg/vectorindex" + "github.com/matrixorigin/matrixone/pkg/vectorindex/hnsw" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" + "github.com/matrixorigin/matrixone/pkg/vm/engine" ) /* IndexConsumer */ @@ -36,44 +41,30 @@ type IndexEntry struct { indexes []*plan.IndexDef } -var sqlExecutorFactory = _sqlExecutorFactory - -func _sqlExecutorFactory(cnUUID string) (executor.SQLExecutor, error) { - // sql executor - v, ok := runtime.ServiceRuntime(cnUUID).GetGlobalVariables(runtime.InternalSQLExecutor) - if !ok { - //os.Stderr.WriteString(fmt.Sprintf("sql executor create failed. cnUUID = %s\n", cnUUID)) - return nil, moerr.NewNotSupportedNoCtx("no implement sqlExecutor") - } - exec := v.(executor.SQLExecutor) - return exec, nil -} - /* IndexConsumer */ type IndexConsumer struct { cnUUID string + cnEngine engine.Engine + cnTxnClient client.TxnClient jobID JobID info *ConsumerInfo tableDef *plan.TableDef sqlWriter IndexSqlWriter - exec executor.SQLExecutor rowdata []any rowdelete []any sqlBufSendCh chan []byte + algo string } var _ Consumer = new(IndexConsumer) func NewIndexConsumer(cnUUID string, + cnEngine engine.Engine, + cnTxnClient client.TxnClient, tableDef *plan.TableDef, jobID JobID, info *ConsumerInfo) (Consumer, error) { - exec, err := sqlExecutorFactory(cnUUID) - if err != nil { - return nil, err - } - ie := &IndexEntry{indexes: make([]*plan.IndexDef, 0, 3)} for _, idx := range tableDef.Indexes { @@ -95,21 +86,22 @@ func NewIndexConsumer(cnUUID string, } c := &IndexConsumer{cnUUID: cnUUID, - jobID: jobID, - info: info, - tableDef: tableDef, - sqlWriter: sqlwriter, - exec: exec, - rowdata: make([]any, len(tableDef.Cols)), - rowdelete: make([]any, 1), + cnEngine: cnEngine, + cnTxnClient: cnTxnClient, + jobID: jobID, + info: info, + tableDef: tableDef, + sqlWriter: sqlwriter, + rowdata: make([]any, len(tableDef.Cols)), + rowdelete: make([]any, 1), + algo: ie.algo, //sqlBufSendCh: make(chan []byte), } return c, nil } -func (c *IndexConsumer) run(ctx context.Context, errch chan error, r DataRetriever) { - +func runIndex(c *IndexConsumer, ctx context.Context, errch chan error, r DataRetriever) { datatype := r.GetDataType() if datatype == ISCPDataType_Snapshot { @@ -125,58 +117,182 @@ func (c *IndexConsumer) run(ctx context.Context, errch chan error, r DataRetriev if !ok { return } - func() { - newctx, cancel := context.WithTimeout(ctx, time.Hour) - - defer cancel() - //os.Stderr.WriteString("Wait for BEGIN but sql. execute anyway\n") - opts := executor.Options{}.WithAccountID(r.GetAccountID()) - res, err := c.exec.Exec(newctx, string(sql), opts) - if err != nil { - logutil.Errorf("cdc indexConsumer(%v) send sql failed, err: %v, sql: %s", c.info, err, string(sql)) - os.Stderr.WriteString(fmt.Sprintf("sql executor run failed. %s\n", string(sql))) - os.Stderr.WriteString(fmt.Sprintf("err :%v\n", err)) - errch <- err - } - res.Close() - }() + + // no transaction required and commit every time. + err := sqlexec.RunTxnWithSqlContext(ctx, c.cnEngine, c.cnTxnClient, c.cnUUID, r.GetAccountID(), 5*time.Minute, nil, nil, + func(sqlproc *sqlexec.SqlProcess, cbdata any) (err error) { + sqlctx := sqlproc.SqlCtx + + res, err := ExecWithResult(sqlproc.GetContext(), string(sql), sqlctx.GetService(), sqlctx.Txn()) + if err != nil { + logutil.Errorf("cdc indexConsumer(%v) send sql failed, err: %v, sql: %s", c.info, err, string(sql)) + os.Stderr.WriteString(fmt.Sprintf("sql executor run failed. %s\n", string(sql))) + os.Stderr.WriteString(fmt.Sprintf("err :%v\n", err)) + return err + } + res.Close() + return nil + }) + + if err != nil { + errch <- err + return + } } } } else { - // TAIL - newctx, cancel := context.WithTimeout(ctx, time.Hour) - defer cancel() - opts := executor.Options{} - err := c.exec.ExecTxn(newctx, - func(exec executor.TxnExecutor) error { + + // all updates under same transaction and transaction can last very long so set timeout to 24 hours + err := sqlexec.RunTxnWithSqlContext(ctx, c.cnEngine, c.cnTxnClient, c.cnUUID, r.GetAccountID(), 24*time.Hour, nil, nil, + func(sqlproc *sqlexec.SqlProcess, cbdata any) (err error) { + sqlctx := sqlproc.SqlCtx + + // TAIL for { select { case <-ctx.Done(): - return nil + return case e2 := <-errch: - return e2 + errch <- e2 + return case sql, ok := <-c.sqlBufSendCh: if !ok { // channel closed - return r.UpdateWatermark(exec, opts.StatementOption()) + return r.UpdateWatermark(sqlproc.GetContext(), sqlctx.GetService(), sqlctx.Txn()) } // update SQL - res, err := exec.Exec(string(sql), opts.StatementOption().WithAccountID(r.GetAccountID())) + var res executor.Result + res, err = ExecWithResult(sqlproc.GetContext(), string(sql), sqlctx.GetService(), sqlctx.Txn()) if err != nil { return err } res.Close() } } - }, opts) + }) + if err != nil { errch <- err return } } +} + +func runHnsw[T types.RealNumbers](c *IndexConsumer, ctx context.Context, errch chan error, r DataRetriever) { + + datatype := r.GetDataType() + + // Suppose we shoult not use transaction here for Snapshot type and commit every time a new batch comes. + // However, HNSW only run in local without save to database until Sync.Save(). + // HNSW is okay to have similar implementation to TAIL + + var err error + var sync *hnsw.HnswSync[T] + + // read-only sql so no need transaction here. All models are loaded at startup. + err = sqlexec.RunTxnWithSqlContext(ctx, c.cnEngine, c.cnTxnClient, c.cnUUID, r.GetAccountID(), 30*time.Minute, nil, nil, + func(sqlproc *sqlexec.SqlProcess, cbdata any) (err error) { + + w := c.sqlWriter.(*HnswSqlWriter[T]) + sync, err = w.NewSync(sqlproc) + return err + }) + + if err != nil { + errch <- err + return + } + if sync == nil { + errch <- moerr.NewInternalErrorNoCtx("failed create HnswSync") + return + } + + defer func() { + if sync != nil { + sync.Destroy() + } + }() + + for { + select { + case <-ctx.Done(): + return + case e2 := <-errch: + errch <- e2 + return + case sql, ok := <-c.sqlBufSendCh: + if !ok { + // channel closed + + // we need a transaction here to save model files and update watermark + err = sqlexec.RunTxnWithSqlContext(ctx, c.cnEngine, c.cnTxnClient, c.cnUUID, r.GetAccountID(), time.Hour, nil, nil, + func(sqlproc *sqlexec.SqlProcess, cbdata any) (err error) { + sqlctx := sqlproc.SqlCtx + + // save model to db + err = sync.Save(sqlproc) + if err != nil { + return + } + + // update watermark + if datatype == ISCPDataType_Tail { + err = r.UpdateWatermark(sqlproc.GetContext(), sqlctx.GetService(), sqlctx.Txn()) + if err != nil { + return + } + } + return + + }) + + if err != nil { + errch <- err + return + } + return + } + + // sql -> cdc + var cdc vectorindex.VectorIndexCdc[T] + err = sonic.Unmarshal(sql, &cdc) + if err != nil { + errch <- err + return + } + + // HNSW models are already in local so hnsw Update should not require executing SQL or should be read-only. No transaction required. + err = sqlexec.RunTxnWithSqlContext(ctx, c.cnEngine, c.cnTxnClient, c.cnUUID, r.GetAccountID(), 30*time.Minute, nil, nil, + func(sqlproc *sqlexec.SqlProcess, cbdata any) (err error) { + return sync.Update(sqlproc, &cdc) + }) + + if err != nil { + errch <- err + return + } + + } + } + +} + +func (c *IndexConsumer) run(ctx context.Context, errch chan error, r DataRetriever) { + + switch c.sqlWriter.(type) { + case *HnswSqlWriter[float32]: + // init HnswSync[float32] + runHnsw[float32](c, ctx, errch, r) + case *HnswSqlWriter[float64]: + // init HnswSync[float64] + runHnsw[float64](c, ctx, errch, r) + default: + // run fulltext/ivfflat index + runIndex(c, ctx, errch, r) + } } func (c *IndexConsumer) processISCPData(ctx context.Context, data *ISCPData, datatype int8, errch chan error) bool { diff --git a/pkg/iscp/index_consumer_test.go b/pkg/iscp/index_consumer_test.go index 2096046ce9adb..a0c2d8f0395b4 100644 --- a/pkg/iscp/index_consumer_test.go +++ b/pkg/iscp/index_consumer_test.go @@ -16,19 +16,17 @@ package iscp import ( "context" - "errors" - "fmt" - "strings" "testing" "github.com/matrixorigin/matrixone/pkg/catalog" - "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/testutil" + "github.com/matrixorigin/matrixone/pkg/testutil/testengine" "github.com/matrixorigin/matrixone/pkg/txn/client" "github.com/matrixorigin/matrixone/pkg/util/executor" "github.com/prashantv/gostub" @@ -56,7 +54,7 @@ func (r *MockRetriever) Next() *ISCPData { return d } -func (r *MockRetriever) UpdateWatermark(executor.TxnExecutor, executor.StatementOption) error { +func (r *MockRetriever) UpdateWatermark(ctx context.Context, cnUUID string, txn client.TxnOperator) error { logutil.Infof("TxnRetriever.UpdateWatermark()") return nil } @@ -115,6 +113,64 @@ func newTestTableDef(pkName string, pkType types.T, vecColName string, vecType t } } +func newTestIvfTableDef(pkName string, pkType types.T, vecColName string, vecType types.T, vecWidth int32) *plan.TableDef { + return &plan.TableDef{ + Name: "test_orig_tbl", + Name2ColIndex: map[string]int32{ + pkName: 0, + vecColName: 1, + "dummy": 2, // Add another col to make sure pk/vec col indices are used + }, + Cols: []*plan.ColDef{ + {Name: pkName, Typ: plan.Type{Id: int32(pkType)}}, + {Name: vecColName, Typ: plan.Type{Id: int32(vecType), Width: vecWidth}}, + {Name: "dummy", Typ: plan.Type{Id: int32(types.T_int32)}}, + }, + Pkey: &plan.PrimaryKeyDef{ + Names: []string{pkName}, + PkeyColName: pkName, + }, + Indexes: []*plan.IndexDef{ + { + IndexName: "ivf_idx", + TableExist: true, + IndexAlgo: catalog.MoIndexIvfFlatAlgo.ToString(), + IndexAlgoTableType: catalog.SystemSI_IVFFLAT_TblType_Metadata, + IndexTableName: "meta_tbl", + Parts: []string{vecColName}, + IndexAlgoParams: `{"lists":"16","op_type":"vector_l2_ops"}`, + }, + { + IndexName: "ivf_idx", + TableExist: true, + IndexAlgo: catalog.MoIndexIvfFlatAlgo.ToString(), + IndexAlgoTableType: catalog.SystemSI_IVFFLAT_TblType_Centroids, + IndexTableName: "centriods", + Parts: []string{vecColName}, + IndexAlgoParams: `{"lists":"16","op_type":"vector_l2_ops"}`, + }, + { + IndexName: "ivf_idx", + TableExist: true, + IndexAlgo: catalog.MoIndexIvfFlatAlgo.ToString(), + IndexAlgoTableType: catalog.SystemSI_IVFFLAT_TblType_Entries, + IndexTableName: "entries", + Parts: []string{vecColName}, + IndexAlgoParams: `{"lists":"16","op_type":"vector_l2_ops"}`, + }, + }, + } +} + +func newTestIvfConsumerInfo() *ConsumerInfo { + return &ConsumerInfo{ + ConsumerType: 0, + DBName: "test_db", + TableName: "test_tbl", + IndexName: "ivf_idx", + } +} + func newTestConsumerInfo() *ConsumerInfo { return &ConsumerInfo{ ConsumerType: 0, @@ -132,83 +188,16 @@ func newTestJobID() JobID { } } -type MockSQLExecutor struct { - txnexec executor.TxnExecutor - sqls []string -} - -// Exec exec a sql in a exists txn. -func (exec *MockSQLExecutor) Exec(ctx context.Context, sql string, opts executor.Options) (executor.Result, error) { - exec.sqls = append(exec.sqls, sql) - //fmt.Printf("Exec %p %v\n", exec.sqls, exec.sqls) - return executor.Result{}, nil -} - -var _ executor.SQLExecutor = new(MockSQLExecutor) - -func mockSqlExecutorFactory(uuid string) (executor.SQLExecutor, error) { - return &MockSQLExecutor{}, nil -} - -type MockErrorTxnExecutor struct { - database string - ctx context.Context - sqls []string -} - -func (exec *MockErrorTxnExecutor) Use(db string) { - exec.database = db -} - -func (exec *MockErrorTxnExecutor) Exec( - sql string, - statementOption executor.StatementOption, -) (executor.Result, error) { - if strings.Contains(sql, "FAILSQL") { - return executor.Result{}, moerr.NewInternalErrorNoCtx("db error") - } else if strings.Contains(sql, "MULTI_ERROR_NO_ROLLBACK") { - var errs error - errs = errors.Join(errs, moerr.NewInternalErrorNoCtx("db error")) - errs = errors.Join(errs, moerr.NewInternalErrorNoCtx("db error 2")) - return executor.Result{}, errs - } else if strings.Contains(sql, "MULTI_ERROR_ROLLBACK") { - var errs error - errs = errors.Join(errs, moerr.NewInternalErrorNoCtx("db error")) - errs = errors.Join(errs, moerr.NewQueryInterrupted(exec.ctx)) - return executor.Result{}, errs - } - - exec.sqls = append(exec.sqls, sql) - //fmt.Printf("APPEND %s", sql) - return executor.Result{}, nil -} - -func (exec *MockErrorTxnExecutor) LockTable(table string) error { - return nil -} - -func (exec *MockErrorTxnExecutor) Txn() client.TxnOperator { - return nil -} - -// ExecTxn executor sql in a txn. execFunc can use TxnExecutor to exec multiple sql -// in a transaction. -// NOTE: Pass SQL stmts one by one to TxnExecutor.Exec(). If you pass multiple SQL stmts to -// TxnExecutor.Exec() as `\n` seperated string, it will only execute the first SQL statement causing Bug. -func (exec *MockSQLExecutor) ExecTxn(ctx context.Context, execFunc func(txn executor.TxnExecutor) error, opts executor.Options) error { - exec.txnexec = &MockErrorTxnExecutor{ctx: ctx} - err := execFunc(exec.txnexec) - exec.sqls = exec.txnexec.(*MockErrorTxnExecutor).sqls - //fmt.Printf("ExecTxn %v\n", exec.sqls) - return err -} - func TestConsumer(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - sqlexecstub := gostub.Stub(&sqlExecutorFactory, mockSqlExecutorFactory) - defer sqlexecstub.Reset() + sqls := make([]string, 0, 1) + stub1 := gostub.Stub(&ExecWithResult, func(_ context.Context, sql string, _ string, _ client.TxnOperator) (executor.Result, error) { + sqls = append(sqls, sql) + return executor.Result{}, nil + }) + defer stub1.Reset() r := &MockRetriever{ insertBatch: nil, @@ -217,34 +206,49 @@ func TestConsumer(t *testing.T) { dtype: ISCPDataType_Snapshot, } - tblDef := newTestTableDef("pk", types.T_int64, "vec", types.T_array_float32, 4) + tblDef := newTestIvfTableDef("pk", types.T_int64, "vec", types.T_array_float32, 4) cnUUID := "a-b-c-d" - info := newTestConsumerInfo() + info := newTestIvfConsumerInfo() job := newTestJobID() - consumer, err := NewConsumer(cnUUID, tblDef, job, info) + catalog.SetupDefines("") + cnEngine, cnClient, _ := testengine.New(ctx) + + consumer, err := NewConsumer(cnUUID, cnEngine, cnClient, tblDef, job, info) require.NoError(t, err) err = consumer.Consume(ctx, r) require.NoError(t, err) } -func TestHnswSnapshot(t *testing.T) { +func TestIvfSnapshot(t *testing.T) { proc := testutil.NewProcess(t) - ctx, cancel := context.WithCancel(context.Background()) + ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, catalog.System_Account) + ctx, cancel := context.WithCancel(ctx) defer cancel() - sqlexecstub := gostub.Stub(&sqlExecutorFactory, mockSqlExecutorFactory) - defer sqlexecstub.Reset() + sqls := make([]string, 0, 1) + stub1 := gostub.Stub(&ExecWithResult, func(_ context.Context, sql string, _ string, _ client.TxnOperator) (executor.Result, error) { + sqls = append(sqls, sql) + return executor.Result{}, nil + }) + defer stub1.Reset() - tblDef := newTestTableDef("pk", types.T_int64, "vec", types.T_array_float32, 2) + /* + sqlexecstub := gostub.Stub(&sqlExecutorFactory, mockSqlExecutorFactory) + defer sqlexecstub.Reset() + */ + + tblDef := newTestIvfTableDef("pk", types.T_int64, "vec", types.T_array_float32, 2) cnUUID := "a-b-c-d" - info := newTestConsumerInfo() + info := newTestIvfConsumerInfo() job := newTestJobID() + catalog.SetupDefines("") + cnEngine, cnClient, _ := testengine.New(ctx) t.Run("snapshot", func(t *testing.T) { - consumer, err := NewConsumer(cnUUID, tblDef, job, info) + consumer, err := NewConsumer(cnUUID, cnEngine, cnClient, tblDef, job, info) require.NoError(t, err) bat := testutil.NewBatchWithVectors( @@ -270,16 +274,19 @@ func TestHnswSnapshot(t *testing.T) { } err = consumer.Consume(ctx, output) require.NoError(t, err) - sqls := consumer.(*IndexConsumer).exec.(*MockSQLExecutor).sqls require.Equal(t, len(sqls), 1) sql := sqls[0] - require.Equal(t, string(sql), `SELECT hnsw_cdc_update('test_db', 'test_tbl', 224, 2, '{"cdc":[{"t":"U","pk":1,"v":[0.1,0.2]},{"t":"U","pk":2,"v":[0.3,0.4]}]}');`) + expected := "REPLACE INTO `test_db`.`entries` (`__mo_index_centroid_fk_version`, `__mo_index_centroid_fk_id`, `__mo_index_pri_col`, `__mo_index_centroid_fk_entry`) WITH centroid as (SELECT * FROM `test_db`.`centriods` WHERE `__mo_index_centroid_version` = (SELECT CAST(__mo_index_val as BIGINT) FROM `test_db`.`meta_tbl` WHERE `__mo_index_key` = 'version') ), src as (SELECT CAST(column_0 as BIGINT) as `src0`, CAST(column_1 as VECF32(2)) as `src1` FROM (VALUES ROW(1,'[0.1, 0.2]'),ROW(2,'[0.3, 0.4]'))) SELECT `__mo_index_centroid_version`, `__mo_index_centroid_id`, src0, src1 FROM src CENTROIDX('vector_l2_ops') JOIN centroid using (`__mo_index_centroid`, `src1`)" + + require.Equal(t, string(sql), expected) + sqls = sqls[:0] //fmt.Printf("Consume %p %v\n", consumer.(*IndexConsumer).exec.(*MockSQLExecutor).sqls, consumer.(*IndexConsumer).exec.(*MockSQLExecutor).sqls) }) t.Run("noMoreData", func(t *testing.T) { - consumer, err := NewConsumer(cnUUID, tblDef, job, info) + sqls = sqls[:0] + consumer, err := NewConsumer(cnUUID, cnEngine, cnClient, tblDef, job, info) require.NoError(t, err) output := &MockRetriever{ @@ -290,27 +297,32 @@ func TestHnswSnapshot(t *testing.T) { } err = consumer.Consume(ctx, output) require.NoError(t, err) - sqls := consumer.(*IndexConsumer).exec.(*MockSQLExecutor).sqls require.Equal(t, len(sqls), 0) }) } -func TestHnswTail(t *testing.T) { +func TestIvfTail(t *testing.T) { proc := testutil.NewProcess(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - sqlexecstub := gostub.Stub(&sqlExecutorFactory, mockSqlExecutorFactory) - defer sqlexecstub.Reset() + sqls := make([]string, 0, 1) + stub1 := gostub.Stub(&ExecWithResult, func(_ context.Context, sql string, _ string, _ client.TxnOperator) (executor.Result, error) { + sqls = append(sqls, sql) + return executor.Result{}, nil + }) + defer stub1.Reset() - tblDef := newTestTableDef("pk", types.T_int64, "vec", types.T_array_float32, 2) + tblDef := newTestIvfTableDef("pk", types.T_int64, "vec", types.T_array_float32, 2) cnUUID := "a-b-c-d" - info := newTestConsumerInfo() + info := newTestIvfConsumerInfo() job := newTestJobID() + catalog.SetupDefines("") + cnEngine, cnClient, _ := testengine.New(ctx) - consumer, err := NewConsumer(cnUUID, tblDef, job, info) + consumer, err := NewConsumer(cnUUID, cnEngine, cnClient, tblDef, job, info) require.NoError(t, err) bat := testutil.NewBatchWithVectors( @@ -357,25 +369,65 @@ func TestHnswTail(t *testing.T) { err = consumer.Consume(ctx, output) require.NoError(t, err) - sqls := consumer.(*IndexConsumer).exec.(*MockSQLExecutor).txnexec.(*MockErrorTxnExecutor).sqls - require.Equal(t, len(sqls), 1) - fmt.Printf("RES: %v\n", sqls) + //fmt.Printf("RES: %v\n", sqls) + require.Equal(t, len(sqls), 2) row1 := []any{int64(1), []float32{0.1, 0.2}} row2 := []any{int64(2), []float32{0.3, 0.4}} - writer, _ := NewHnswSqlWriter("hnsw", job, info, tblDef, tblDef.Indexes) + writer, err := NewIvfflatSqlWriter("ivfflat", job, info, tblDef, tblDef.Indexes) + require.NoError(t, err) writer.Insert(ctx, row1) writer.Insert(ctx, row2) writer.Delete(ctx, row1) writer.Delete(ctx, row2) - /* - cdcForJson.Start = "1-0" - cdcForJson.End = "2-0" - */ expectedSqlBytes, _ := writer.ToSql() + //fmt.Printf("EEEEEEEEEEEE %s %s\n", expectedSqlBytes, sqls[0]) require.Equal(t, string(expectedSqlBytes), sqls[0]) } + +func TestHnswConsumer(t *testing.T) { + + //proc := testutil.NewProcess(t) + + ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, catalog.System_Account) + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + sqls := make([]string, 0, 1) + stub1 := gostub.Stub(&ExecWithResult, func(_ context.Context, sql string, _ string, _ client.TxnOperator) (executor.Result, error) { + sqls = append(sqls, sql) + return executor.Result{}, nil + }) + defer stub1.Reset() + + tblDef := newTestTableDef("pk", types.T_int64, "vec", types.T_array_float32, 2) + info := newTestConsumerInfo() + job := newTestJobID() + /* + cnUUID := "a-b-c-d" + catalog.SetupDefines("") + cnEngine, cnClient, _ := testengine.New(ctx) + */ + + t.Run("HnswSqlWriter", func(t *testing.T) { + + row1 := []any{int64(1), []float32{0.1, 0.2}} + row2 := []any{int64(2), []float32{0.3, 0.4}} + + writer, err := NewHnswSqlWriter("hnsw", job, info, tblDef, tblDef.Indexes) + require.NoError(t, err) + writer.Insert(ctx, row1) + writer.Insert(ctx, row2) + writer.Delete(ctx, row1) + writer.Delete(ctx, row2) + json, _ := writer.ToSql() + + expectedSqlBytes := `{"cdc":[{"t":"I","pk":1,"v":[0.1,0.2]},{"t":"I","pk":2,"v":[0.3,0.4]},{"t":"D","pk":1},{"t":"D","pk":2}]}` + require.Equal(t, expectedSqlBytes, string(json)) + + }) +} diff --git a/pkg/iscp/index_sqlwriter.go b/pkg/iscp/index_sqlwriter.go index 5242a633cdac7..e25db1a9936f5 100644 --- a/pkg/iscp/index_sqlwriter.go +++ b/pkg/iscp/index_sqlwriter.go @@ -25,6 +25,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/vectorindex" + "github.com/matrixorigin/matrixone/pkg/vectorindex/hnsw" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" ) const ( @@ -339,8 +341,9 @@ func NewGenericHnswSqlWriter[T types.RealNumbers](algo string, jobID JobID, info // get the first indexdef as they are the same idxdef := indexdef[0] + writer_capacity := 8192 - w := &HnswSqlWriter[T]{tabledef: tabledef, indexdef: indexdef, jobID: jobID, info: info, cdc: vectorindex.NewVectorIndexCdc[T]()} + w := &HnswSqlWriter[T]{tabledef: tabledef, indexdef: indexdef, jobID: jobID, info: info, cdc: vectorindex.NewVectorIndexCdc[T](writer_capacity)} paramstr := idxdef.IndexAlgoParams var meta, storage string @@ -512,10 +515,12 @@ func (w *HnswSqlWriter[T]) ToSql() ([]byte, error) { if err != nil { return nil, err } - // pad extra space at the front and send SQL - sql := fmt.Sprintf("SELECT hnsw_cdc_update('%s', '%s', %d, %d, '%s');", w.meta.DbName, w.meta.Table, w.meta.VecType, w.meta.Dimension, js) - return []byte(sql), nil + return []byte(js), nil +} + +func (w *HnswSqlWriter[T]) NewSync(sqlproc *sqlexec.SqlProcess) (*hnsw.HnswSync[T], error) { + return hnsw.NewHnswSync[T](sqlproc, w.meta.DbName, w.meta.Table, w.meta.VecType, w.meta.Dimension) } // Implementation of Ivfflat Sql writer diff --git a/pkg/iscp/iteration.go b/pkg/iscp/iteration.go index a28a8639003f1..4cea322e922c0 100644 --- a/pkg/iscp/iteration.go +++ b/pkg/iscp/iteration.go @@ -151,7 +151,7 @@ func ExecuteIteration( DBName: dbName, TableName: tableName, } - consumers[i], err = NewConsumer(cnUUID, rel.CopyTableDef(ctx), jobID, &jobSpecs[i].ConsumerInfo) + consumers[i], err = NewConsumer(cnUUID, cnEngine, cnTxnClient, rel.CopyTableDef(ctx), jobID, &jobSpecs[i].ConsumerInfo) if err != nil { return } diff --git a/pkg/iscp/mock_consumer.go b/pkg/iscp/mock_consumer.go index b7eea4aa0693a..ace1da2e6b193 100644 --- a/pkg/iscp/mock_consumer.go +++ b/pkg/iscp/mock_consumer.go @@ -28,7 +28,9 @@ import ( "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/objectio" "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/txn/client" "github.com/matrixorigin/matrixone/pkg/util/executor" + "github.com/matrixorigin/matrixone/pkg/vm/engine" ) const ( @@ -43,6 +45,9 @@ const ( var _ Consumer = &interalSqlConsumer{} type interalSqlConsumer struct { + cnUUID string + cnEngine engine.Engine + cnTxnClient client.TxnClient internalSqlExecutor executor.SQLExecutor jobName string @@ -94,13 +99,18 @@ type interalSqlConsumer struct { func NewInteralSqlConsumer( cnUUID string, + cnEngine engine.Engine, + cnTxnClient client.TxnClient, tableDef *plan.TableDef, jobID JobID, info *ConsumerInfo, ) (Consumer, error) { s := &interalSqlConsumer{ - tableInfo: tableDef, - jobName: jobID.JobName, + tableInfo: tableDef, + jobName: jobID.JobName, + cnUUID: cnUUID, + cnEngine: cnEngine, + cnTxnClient: cnTxnClient, } s.maxAllowedPacket = uint64(1024 * 1024) logutil.Infof("iscp mysqlSinker(%v) maxAllowedPacket = %d", tableDef.Name, s.maxAllowedPacket) @@ -230,25 +240,39 @@ func (s *interalSqlConsumer) Consume(ctx context.Context, data DataRetriever) er ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, catalog.System_Account) ctx, cancel := context.WithTimeout(ctx, time.Minute*5) defer cancel() - err := s.internalSqlExecutor.ExecTxn(ctx, func(txn executor.TxnExecutor) error { - for { - iscpData := data.Next() - noMoreData, err := s.consumeData(ctx, data.GetDataType(), iscpData, txn) - if err != nil { - return err - } - if noMoreData { - return nil - } + txn, err := getTxn(ctx, s.cnEngine, s.cnTxnClient, "internalSqlConsumer") + if err != nil { + return err + } + + defer func() { + if err != nil { + err = txn.Rollback(ctx) + } else { + err = txn.Commit(ctx) } - }, executor.Options{}) - return err + + if err != nil { + logutil.Error("InteralSqlConsumer Consume Tail failed") + } + }() + + for { + iscpData := data.Next() + noMoreData, err := s.consumeData(ctx, data.GetDataType(), iscpData, txn) + if err != nil { + return err + } + if noMoreData { + return nil + } + } default: panic("logic error") } } -func (s *interalSqlConsumer) consumeData(ctx context.Context, dataType int8, iscpData *ISCPData, txn executor.TxnExecutor) (noMoreData bool, err error) { +func (s *interalSqlConsumer) consumeData(ctx context.Context, dataType int8, iscpData *ISCPData, txn client.TxnOperator) (noMoreData bool, err error) { if iscpData == nil { noMoreData = true return @@ -263,7 +287,7 @@ func (s *interalSqlConsumer) consumeData(ctx context.Context, dataType int8, isc if noMoreData { if s.dataRetriever.GetDataType() != ISCPDataType_Snapshot { - err = s.dataRetriever.UpdateWatermark(txn, executor.StatementOption{}) + err = s.dataRetriever.UpdateWatermark(ctx, s.cnUUID, txn) if err != nil { return } @@ -310,12 +334,12 @@ func (s *interalSqlConsumer) sinkSnapshot(ctx context.Context, bat *AtomicBatch) } // step3: append to sqlBuf, send sql if sqlBuf is full - if sqlBuffer, err = s.appendSqlBuf(UpsertRow, sqlBuffer, nil); err != nil { + if sqlBuffer, err = s.appendSqlBuf(ctx, UpsertRow, sqlBuffer, nil); err != nil { panic(err) } } } - err = s.tryFlushSqlBuf(nil, sqlBuffer) + err = s.tryFlushSqlBuf(ctx, nil, sqlBuffer) if err != nil { return err } @@ -324,7 +348,7 @@ func (s *interalSqlConsumer) sinkSnapshot(ctx context.Context, bat *AtomicBatch) // insertBatch and deleteBatch is sorted by ts // for the same ts, delete first, then insert -func (s *interalSqlConsumer) sinkTail(ctx context.Context, insertBatch, deleteBatch *AtomicBatch, txn executor.TxnExecutor) error { +func (s *interalSqlConsumer) sinkTail(ctx context.Context, insertBatch, deleteBatch *AtomicBatch, txn client.TxnOperator) error { var err error insertIter := insertBatch.GetRowIterator().(*atomicBatchRowIter) @@ -350,7 +374,7 @@ func (s *interalSqlConsumer) sinkTail(ctx context.Context, insertBatch, deleteBa // get next item deleteIterHasNext = deleteIter.Next() } - err = s.tryFlushSqlBuf(txn, sqlBuffer) + err = s.tryFlushSqlBuf(ctx, txn, sqlBuffer) if err != nil { return err } @@ -367,14 +391,14 @@ func (s *interalSqlConsumer) sinkTail(ctx context.Context, insertBatch, deleteBa // get next item insertIterHasNext = insertIter.Next() } - err = s.tryFlushSqlBuf(txn, sqlBuffer) + err = s.tryFlushSqlBuf(ctx, txn, sqlBuffer) if err != nil { return err } return nil } -func (s *interalSqlConsumer) sinkInsert(ctx context.Context, insertIter *atomicBatchRowIter, sqlBuffer []byte, txn executor.TxnExecutor) (res []byte, err error) { +func (s *interalSqlConsumer) sinkInsert(ctx context.Context, insertIter *atomicBatchRowIter, sqlBuffer []byte, txn client.TxnOperator) (res []byte, err error) { // if last row is not insert row, need complete the last sql first if s.preRowType != InsertRow { panic("logic error") @@ -391,14 +415,14 @@ func (s *interalSqlConsumer) sinkInsert(ctx context.Context, insertIter *atomicB } // step3: append to sqlBuf - if res, err = s.appendSqlBuf(InsertRow, sqlBuffer, txn); err != nil { + if res, err = s.appendSqlBuf(ctx, InsertRow, sqlBuffer, txn); err != nil { return } return } -func (s *interalSqlConsumer) sinkDelete(ctx context.Context, deleteIter *atomicBatchRowIter, sqlBuffer []byte, txn executor.TxnExecutor) (res []byte, err error) { +func (s *interalSqlConsumer) sinkDelete(ctx context.Context, deleteIter *atomicBatchRowIter, sqlBuffer []byte, txn client.TxnOperator) (res []byte, err error) { // if last row is not insert row, need complete the last sql first if s.preRowType != DeleteRow { panic("logic error") @@ -415,14 +439,14 @@ func (s *interalSqlConsumer) sinkDelete(ctx context.Context, deleteIter *atomicB } // step3: append to sqlBuf - if res, err = s.appendSqlBuf(DeleteRow, sqlBuffer, txn); err != nil { + if res, err = s.appendSqlBuf(ctx, DeleteRow, sqlBuffer, txn); err != nil { return } return } -func (s *interalSqlConsumer) tryFlushSqlBuf(txn executor.TxnExecutor, sqlBuffer []byte) (err error) { +func (s *interalSqlConsumer) tryFlushSqlBuf(ctx context.Context, txn client.TxnOperator, sqlBuffer []byte) (err error) { if len(sqlBuffer) == 0 { return } @@ -440,7 +464,7 @@ func (s *interalSqlConsumer) tryFlushSqlBuf(txn executor.TxnExecutor, sqlBuffer return } var result executor.Result - result, err = txn.Exec(string(sqlBuffer), executor.StatementOption{}) + result, err = ExecWithResult(ctx, string(sqlBuffer), s.cnUUID, txn) result.Close() if err != nil { logutil.Errorf("iscp interalSqlConsumer(%v) send sql failed, err: %v, sql: %s", s.tableInfo.Name, err, sqlBuffer[:]) @@ -452,7 +476,7 @@ func (s *interalSqlConsumer) tryFlushSqlBuf(txn executor.TxnExecutor, sqlBuffer // appendSqlBuf appends rowBuf to sqlBuf if not exceed its cap // otherwise, send sql to downstream first, then reset sqlBuf and append -func (s *interalSqlConsumer) appendSqlBuf(rowType RowType, sqlBuffer []byte, txn executor.TxnExecutor) (res []byte, err error) { +func (s *interalSqlConsumer) appendSqlBuf(ctx context.Context, rowType RowType, sqlBuffer []byte, txn client.TxnOperator) (res []byte, err error) { suffixLen := len(s.insertSuffix) if rowType == DeleteRow { suffixLen = len(s.deleteSuffix) @@ -486,7 +510,7 @@ func (s *interalSqlConsumer) appendSqlBuf(rowType RowType, sqlBuffer []byte, txn if s.isNonEmptyDeleteStmt(sqlBuffer) { sqlBuffer = appendBytes(sqlBuffer, s.deleteSuffix) } - s.tryFlushSqlBuf(txn, sqlBuffer) + s.tryFlushSqlBuf(ctx, txn, sqlBuffer) sqlBuffer = make([]byte, 0) // reset s.sqlBuf diff --git a/pkg/iscp/types.go b/pkg/iscp/types.go index 690018166c6ed..67b2a3c99af6e 100644 --- a/pkg/iscp/types.go +++ b/pkg/iscp/types.go @@ -27,7 +27,6 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/pb/txn" "github.com/matrixorigin/matrixone/pkg/txn/client" - "github.com/matrixorigin/matrixone/pkg/util/executor" "github.com/matrixorigin/matrixone/pkg/vm/engine" "github.com/matrixorigin/matrixone/pkg/vm/engine/cmd_util" "github.com/matrixorigin/matrixone/pkg/vm/engine/readutil" @@ -54,7 +53,7 @@ const ( type DataRetriever interface { Next() (iscpData *ISCPData) - UpdateWatermark(executor.TxnExecutor, executor.StatementOption) error + UpdateWatermark(ctx context.Context, cnUUID string, txn client.TxnOperator) error GetDataType() int8 GetAccountID() uint32 GetTableID() uint64 diff --git a/pkg/iscp/watermark_updater.go b/pkg/iscp/watermark_updater.go index 53244d9a5934a..54504a8ee27ef 100644 --- a/pkg/iscp/watermark_updater.go +++ b/pkg/iscp/watermark_updater.go @@ -51,7 +51,7 @@ func UnmarshalJobSpec(jsonByte []byte) (*JobSpec, error) { return &jobSpec, nil } -func ExecWithResult( +var ExecWithResult = func( ctx context.Context, sql string, cnUUID string, diff --git a/pkg/partitionservice/service.go b/pkg/partitionservice/service.go index da800c4e7adfe..50cdccb3d7436 100644 --- a/pkg/partitionservice/service.go +++ b/pkg/partitionservice/service.go @@ -113,11 +113,13 @@ func (s *Service) Delete( if txnOp != nil { txnOp.AppendEventCallback( client.CommitEvent, - func(te client.TxnEvent) { - s.mu.Lock() - delete(s.mu.tables, tableID) - s.mu.Unlock() - }, + client.NewTxnEventCallback( + func(ctx context.Context, txnOp client.TxnOperator, te client.TxnEvent, v any) (err error) { + s.mu.Lock() + delete(s.mu.tables, tableID) + s.mu.Unlock() + return + }), ) } diff --git a/pkg/partitionservice/storage_test.go b/pkg/partitionservice/storage_test.go index 1feea540c5e10..693ddc235d50a 100644 --- a/pkg/partitionservice/storage_test.go +++ b/pkg/partitionservice/storage_test.go @@ -116,18 +116,20 @@ func (s *memStorage) Create( txnOp.AppendEventCallback( client.ClosedEvent, - func(txn client.TxnEvent) { - s.Lock() - defer s.Unlock() - - v, ok := s.uncommitted[def.TblId] - if txn.Committed() { - if ok { - s.committed[def.TblId] = v + client.NewTxnEventCallback( + func(ctx context.Context, txnOp client.TxnOperator, txn client.TxnEvent, cbdata any) error { + s.Lock() + defer s.Unlock() + + v, ok := s.uncommitted[def.TblId] + if txn.Committed() { + if ok { + s.committed[def.TblId] = v + } } - } - delete(s.uncommitted, def.TblId) - }, + delete(s.uncommitted, def.TblId) + return nil + }), ) return nil } @@ -154,15 +156,17 @@ func (s *memStorage) Delete( txnOp.AppendEventCallback( client.ClosedEvent, - func(txn client.TxnEvent) { - s.Lock() - defer s.Unlock() - - delete(s.uncommitted, table) - if txn.Committed() { - delete(s.committed, table) - } - }, + client.NewTxnEventCallback( + func(ctx context.Context, txnOp client.TxnOperator, txn client.TxnEvent, v any) error { + s.Lock() + defer s.Unlock() + + delete(s.uncommitted, table) + if txn.Committed() { + delete(s.committed, table) + } + return nil + }), ) return nil } diff --git a/pkg/shardservice/service.go b/pkg/shardservice/service.go index af2bde20eb231..1fa40a7842d82 100644 --- a/pkg/shardservice/service.go +++ b/pkg/shardservice/service.go @@ -181,21 +181,23 @@ func (s *service) Create( if !s.options.disableAppendCreateCallback { txnOp.AppendEventCallback( client.ClosedEvent, - func(txn client.TxnEvent) { - if txn.Committed() { - s.logger.Info("sharding table created", - zap.Uint64("table", table), - zap.String("committed", txn.Txn.CommitTS.DebugString()), - ) - - // The callback here is not guaranteed to execute after the transaction has - // already committed. - // The creation will lazy execute in Read. - s.createC <- table - } else { - s.atomic.abort.Add(1) - } - }, + client.NewTxnEventCallback( + func(ctx context.Context, _txnOp client.TxnOperator, txn client.TxnEvent, v any) error { + if txn.Committed() { + s.logger.Info("sharding table created", + zap.Uint64("table", table), + zap.String("committed", txn.Txn.CommitTS.DebugString()), + ) + + // The callback here is not guaranteed to execute after the transaction has + // already committed. + // The creation will lazy execute in Read. + s.createC <- table + } else { + s.atomic.abort.Add(1) + } + return nil + }), ) } @@ -220,13 +222,15 @@ func (s *service) Delete( if !s.options.disableAppendDeleteCallback { txnOp.AppendEventCallback( client.ClosedEvent, - func(txn client.TxnEvent) { - if txn.Committed() { - s.deleteC <- table - } else { - s.atomic.abort.Add(1) - } - }, + client.NewTxnEventCallback( + func(ctx context.Context, _txnOp client.TxnOperator, txn client.TxnEvent, v any) error { + if txn.Committed() { + s.deleteC <- table + } else { + s.atomic.abort.Add(1) + } + return nil + }), ) } diff --git a/pkg/shardservice/storage_test.go b/pkg/shardservice/storage_test.go index 362dd7e3b796a..f5d18392b2660 100644 --- a/pkg/shardservice/storage_test.go +++ b/pkg/shardservice/storage_test.go @@ -125,15 +125,17 @@ func (s *MemShardStorage) Create( } txnOp.AppendEventCallback( client.ClosedEvent, - func(txn client.TxnEvent) { - s.Lock() - defer s.Unlock() + client.NewTxnEventCallback( + func(ctx context.Context, txnOp client.TxnOperator, txn client.TxnEvent, cbdata any) error { + s.Lock() + defer s.Unlock() - if txn.Committed() { - s.committed[table] = v - } - delete(s.uncommittedAdd, table) - }, + if txn.Committed() { + s.committed[table] = v + } + delete(s.uncommittedAdd, table) + return nil + }), ) return true, nil } @@ -158,15 +160,17 @@ func (s *MemShardStorage) Delete( s.uncommittedDelete[table] = struct{}{} txnOp.AppendEventCallback( client.ClosedEvent, - func(txn client.TxnEvent) { - s.Lock() - defer s.Unlock() + client.NewTxnEventCallback( + func(ctx context.Context, txnOp client.TxnOperator, txn client.TxnEvent, v any) error { + s.Lock() + defer s.Unlock() - if txn.Committed() { - delete(s.committed, table) - } - delete(s.uncommittedDelete, table) - }, + if txn.Committed() { + delete(s.committed, table) + } + delete(s.uncommittedDelete, table) + return nil + }), ) return true, nil } diff --git a/pkg/sql/colexec/table_function/fulltext.go b/pkg/sql/colexec/table_function/fulltext.go index b4cf91e02052c..7e0804389aaae 100644 --- a/pkg/sql/colexec/table_function/fulltext.go +++ b/pkg/sql/colexec/table_function/fulltext.go @@ -240,7 +240,7 @@ func runWordStats( return } - result, err = ft_runSql_streaming(ctx, proc, sql, u.streamCh, u.errCh) + result, err = ft_runSql_streaming(ctx, sqlexec.NewSqlProcess(proc), sql, u.streamCh, u.errCh) return } @@ -414,7 +414,7 @@ func groupby(u *fulltextState, proc *process.Process, s *fulltext.SearchAccum) ( func runCountStar(proc *process.Process, s *fulltext.SearchAccum) (executor.Result, error) { sql := fmt.Sprintf(countstar_sql, s.TblName, fulltext.DOC_LEN_WORD) - res, err := ft_runSql(proc, sql) + res, err := ft_runSql(sqlexec.NewSqlProcess(proc), sql) if err != nil { return executor.Result{}, err } diff --git a/pkg/sql/colexec/table_function/fulltext_test.go b/pkg/sql/colexec/table_function/fulltext_test.go index b6de377a9d220..dfbabf5df1b4b 100644 --- a/pkg/sql/colexec/table_function/fulltext_test.go +++ b/pkg/sql/colexec/table_function/fulltext_test.go @@ -27,6 +27,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/testutil" "github.com/matrixorigin/matrixone/pkg/util/executor" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" "github.com/stretchr/testify/require" @@ -113,19 +114,20 @@ func newFTTestCase(t *testing.T, m *mpool.MPool, attrs []string, algo fulltext.F return ret } -func fake_runSql(proc *process.Process, sql string) (executor.Result, error) { - +func fake_runSql(sqlproc *sqlexec.SqlProcess, sql string) (executor.Result, error) { + proc := sqlproc.Proc // give count return executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{makeCountBatchFT(proc)}}, nil } func fake_runSql_streaming( ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, sql string, ch chan executor.Result, err_chan chan error, ) (executor.Result, error) { + proc := sqlproc.Proc defer close(ch) res := executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{makeTextBatchFT(proc)}} ch <- res diff --git a/pkg/sql/colexec/table_function/hnsw_create.go b/pkg/sql/colexec/table_function/hnsw_create.go index 38aedbbdf2f50..63489b87c44f9 100644 --- a/pkg/sql/colexec/table_function/hnsw_create.go +++ b/pkg/sql/colexec/table_function/hnsw_create.go @@ -76,7 +76,7 @@ func (u *hnswCreateState) end(tf *TableFunction, proc *process.Process) error { } for _, s := range sqls { - res, err := hnsw_runSql(proc, s) + res, err := hnsw_runSql(sqlexec.NewSqlProcess(proc), s) if err != nil { return err } @@ -216,9 +216,9 @@ func (u *hnswCreateState) start(tf *TableFunction, proc *process.Process, nthRow switch u.idxcfg.Usearch.Quantization { case usearch.F32: - u.buildf32, err = hnsw.NewHnswBuild[float32](proc, uid, tf.MaxParallel, u.idxcfg, u.tblcfg) + u.buildf32, err = hnsw.NewHnswBuild[float32](sqlexec.NewSqlProcess(proc), uid, tf.MaxParallel, u.idxcfg, u.tblcfg) case usearch.F64: - u.buildf64, err = hnsw.NewHnswBuild[float64](proc, uid, tf.MaxParallel, u.idxcfg, u.tblcfg) + u.buildf64, err = hnsw.NewHnswBuild[float64](sqlexec.NewSqlProcess(proc), uid, tf.MaxParallel, u.idxcfg, u.tblcfg) } if err != nil { return err diff --git a/pkg/sql/colexec/table_function/hnsw_create_test.go b/pkg/sql/colexec/table_function/hnsw_create_test.go index fed4b9d84098b..2d3467605c8cb 100644 --- a/pkg/sql/colexec/table_function/hnsw_create_test.go +++ b/pkg/sql/colexec/table_function/hnsw_create_test.go @@ -27,6 +27,7 @@ import ( plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/testutil" "github.com/matrixorigin/matrixone/pkg/util/executor" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" "github.com/stretchr/testify/require" @@ -83,8 +84,8 @@ func newHnswCreateTestCase(t *testing.T, m *mpool.MPool, attrs []string, param s return ret } -func mock_hnsw_runSql(proc *process.Process, sql string) (executor.Result, error) { - +func mock_hnsw_runSql(sqlproc *sqlexec.SqlProcess, sql string) (executor.Result, error) { + proc := sqlproc.Proc return executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{}}, nil } diff --git a/pkg/sql/colexec/table_function/hnsw_search.go b/pkg/sql/colexec/table_function/hnsw_search.go index 2c53c0f845a45..4c28fa727f298 100644 --- a/pkg/sql/colexec/table_function/hnsw_search.go +++ b/pkg/sql/colexec/table_function/hnsw_search.go @@ -29,6 +29,7 @@ import ( veccache "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" "github.com/matrixorigin/matrixone/pkg/vectorindex/hnsw" "github.com/matrixorigin/matrixone/pkg/vectorindex/metric" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" usearch "github.com/unum-cloud/usearch/golang" @@ -238,7 +239,7 @@ func runHnswSearch[T types.RealNumbers](proc *process.Process, u *hnswSearchStat algo := newHnswAlgo(u.idxcfg, u.tblcfg) var keys any - keys, u.distances, err = veccache.Cache.Search(proc, u.tblcfg.IndexTable, algo, fa, vectorindex.RuntimeConfig{Limit: uint(u.limit)}) + keys, u.distances, err = veccache.Cache.Search(sqlexec.NewSqlProcess(proc), u.tblcfg.IndexTable, algo, fa, vectorindex.RuntimeConfig{Limit: uint(u.limit)}) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/hnsw_search_test.go b/pkg/sql/colexec/table_function/hnsw_search_test.go index 0ce6356d0d0ab..fba55b4cbba79 100644 --- a/pkg/sql/colexec/table_function/hnsw_search_test.go +++ b/pkg/sql/colexec/table_function/hnsw_search_test.go @@ -29,6 +29,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/vectorindex" "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" veccache "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" "github.com/stretchr/testify/assert" @@ -100,7 +101,7 @@ type MockSearch struct { Tblcfg vectorindex.IndexTableConfig } -func (m *MockSearch) Search(proc *process.Process, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { +func (m *MockSearch) Search(sqlproc *sqlexec.SqlProcess, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { //time.Sleep(2 * time.Millisecond) return []int64{1}, []float64{2.0}, nil } @@ -108,7 +109,7 @@ func (m *MockSearch) Search(proc *process.Process, query any, rt vectorindex.Run func (m *MockSearch) Destroy() { } -func (m *MockSearch) Load(*process.Process) error { +func (m *MockSearch) Load(*sqlexec.SqlProcess) error { //time.Sleep(6 * time.Second) return nil } diff --git a/pkg/sql/colexec/table_function/ivf_create.go b/pkg/sql/colexec/table_function/ivf_create.go index 3f761fbc1ea89..4d58fa8e79c02 100644 --- a/pkg/sql/colexec/table_function/ivf_create.go +++ b/pkg/sql/colexec/table_function/ivf_create.go @@ -76,7 +76,7 @@ func clustering[T types.RealNumbers](u *ivfCreateState, tf *TableFunction, proc var err error var ok bool - version, err := ivfflat.GetVersion(proc, u.tblcfg) + version, err := ivfflat.GetVersion(sqlexec.NewSqlProcess(proc), u.tblcfg) if err != nil { return err } @@ -126,7 +126,7 @@ func clustering[T types.RealNumbers](u *ivfCreateState, tf *TableFunction, proc sqls := []string{sql} for _, s := range sqls { - res, err := ivf_runSql(proc, s) + res, err := ivf_runSql(sqlexec.NewSqlProcess(proc), s) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/ivf_search.go b/pkg/sql/colexec/table_function/ivf_search.go index e06ea3ebb5c59..2ef541a3b9fe3 100644 --- a/pkg/sql/colexec/table_function/ivf_search.go +++ b/pkg/sql/colexec/table_function/ivf_search.go @@ -29,6 +29,7 @@ import ( veccache "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" "github.com/matrixorigin/matrixone/pkg/vectorindex/ivfflat" "github.com/matrixorigin/matrixone/pkg/vectorindex/metric" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -194,7 +195,7 @@ func (u *ivfSearchState) start(tf *TableFunction, proc *process.Process, nthRow u.idxcfg.Type = vectorindex.IVFFLAT // get version - version, err := getVersion(proc, u.tblcfg) + version, err := getVersion(sqlexec.NewSqlProcess(proc), u.tblcfg) if err != nil { return err } @@ -243,7 +244,7 @@ func runIvfSearchVector[T types.RealNumbers](u *ivfSearchState, proc *process.Pr return err } key := fmt.Sprintf("%s:%d", u.tblcfg.IndexTable, u.idxcfg.Ivfflat.Version) - u.keys, u.distances, err = veccache.Cache.Search(proc, key, algo, fa, vectorindex.RuntimeConfig{Limit: uint(u.limit), Probe: uint(u.tblcfg.Nprobe)}) + u.keys, u.distances, err = veccache.Cache.Search(sqlexec.NewSqlProcess(proc), key, algo, fa, vectorindex.RuntimeConfig{Limit: uint(u.limit), Probe: uint(u.tblcfg.Nprobe)}) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/ivf_search_test.go b/pkg/sql/colexec/table_function/ivf_search_test.go index 268be06f9266a..ea0d42631ccae 100644 --- a/pkg/sql/colexec/table_function/ivf_search_test.go +++ b/pkg/sql/colexec/table_function/ivf_search_test.go @@ -30,6 +30,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/vectorindex" "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" veccache "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" "github.com/stretchr/testify/require" @@ -94,12 +95,12 @@ func newIvfSearchTestCase(t *testing.T, m *mpool.MPool, attrs []string, param st return ret } -func mock_ivf_runSql(proc *process.Process, sql string) (executor.Result, error) { - +func mock_ivf_runSql(sqlproc *sqlexec.SqlProcess, sql string) (executor.Result, error) { + proc := sqlproc.Proc return executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{}}, nil } -func mockVersion(proc *process.Process, tblcfg vectorindex.IndexTableConfig) (int64, error) { +func mockVersion(sqlproc *sqlexec.SqlProcess, tblcfg vectorindex.IndexTableConfig) (int64, error) { return 0, nil } @@ -108,7 +109,7 @@ type MockIvfSearch[T types.RealNumbers] struct { Tblcfg vectorindex.IndexTableConfig } -func (m *MockIvfSearch[T]) Search(proc *process.Process, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { +func (m *MockIvfSearch[T]) Search(sqlproc *sqlexec.SqlProcess, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { //time.Sleep(2 * time.Millisecond) return []any{int64(1)}, []float64{2.0}, nil } @@ -116,7 +117,7 @@ func (m *MockIvfSearch[T]) Search(proc *process.Process, query any, rt vectorind func (m *MockIvfSearch[T]) Destroy() { } -func (m *MockIvfSearch[T]) Load(*process.Process) error { +func (m *MockIvfSearch[T]) Load(*sqlexec.SqlProcess) error { //time.Sleep(6 * time.Second) return nil } diff --git a/pkg/sql/colexec/table_function/metadata_scan.go b/pkg/sql/colexec/table_function/metadata_scan.go index 96de146da0967..d3edc630ea9b3 100644 --- a/pkg/sql/colexec/table_function/metadata_scan.go +++ b/pkg/sql/colexec/table_function/metadata_scan.go @@ -16,13 +16,15 @@ package table_function import ( "fmt" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "go.uber.org/zap" + "strings" + "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/perfcounter" "github.com/matrixorigin/matrixone/pkg/sql/colexec" - "strings" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" @@ -71,7 +73,7 @@ func getIndexTableNameByIndexName(proc *process.Process, dbname, tablename, inde logutil.Info("relID", zap.Uint64("value", tableid)) sql := fmt.Sprintf("SELECT distinct(index_table_name) FROM mo_catalog.mo_indexes WHERE table_id = '%d' AND name = '%s'", tableid, indexname) - result, err := sqlexec.RunSql(proc, sql) + result, err := sqlexec.RunSql(sqlexec.NewSqlProcess(proc), sql) if err != nil { return "", err } diff --git a/pkg/sql/compile/alter.go b/pkg/sql/compile/alter.go index 785d2815556c1..74ef6dcc64124 100644 --- a/pkg/sql/compile/alter.go +++ b/pkg/sql/compile/alter.go @@ -49,6 +49,7 @@ func convertDBEOB(ctx context.Context, e error, name string) error { func (s *Scope) AlterTableCopy(c *Compile) error { qry := s.Plan.GetDdl().GetAlterTable() dbName := qry.Database + if dbName == "" { dbName = c.db } @@ -149,25 +150,34 @@ func (s *Scope) AlterTableCopy(c *Compile) error { if qry.Options.SkipPkDedup || len(qry.Options.SkipUniqueIdxDedup) > 0 { opt = opt.WithAlterCopyOpt(qry.Options) } - // 4. copy the original table data to the temporary replica table - err = c.runSqlWithOptions(qry.InsertTmpDataSql, opt) + + //4. obtain relation for new tables + newRel, err := dbSource.Relation(c.proc.Ctx, qry.CopyTableDef.Name, nil) if err != nil { - c.proc.Error(c.proc.Ctx, "insert data to copy table for alter table", + c.proc.Error(c.proc.Ctx, "obtain new relation for copy table for alter table", zap.String("databaseName", dbName), zap.String("origin tableName", qry.GetTableDef().Name), - zap.String("copy tableName", qry.CopyTableDef.Name), - zap.String("InsertTmpDataSql", qry.InsertTmpDataSql), + zap.String("copy table name", qry.CopyTableDef.Name), zap.Error(err)) return err } - //5. obtain relation for new tables - newRel, err := dbSource.Relation(c.proc.Ctx, qry.CopyTableDef.Name, nil) + //5. ISCP: temp table already created pitr and iscp job with temp table name + // and we don't want iscp to run with temp table so drop pitr and iscp job with the temp table here + newTmpTableDef := newRel.CopyTableDef(c.proc.Ctx) + err = DropAllIndexCdcTasks(c, newTmpTableDef, dbName, qry.CopyTableDef.Name) if err != nil { - c.proc.Error(c.proc.Ctx, "obtain new relation for copy table for alter table", + return err + } + + // 6. copy the original table data to the temporary replica table + err = c.runSqlWithOptions(qry.InsertTmpDataSql, opt) + if err != nil { + c.proc.Error(c.proc.Ctx, "insert data to copy table for alter table", zap.String("databaseName", dbName), zap.String("origin tableName", qry.GetTableDef().Name), - zap.String("copy table name", qry.CopyTableDef.Name), + zap.String("copy tableName", qry.CopyTableDef.Name), + zap.String("InsertTmpDataSql", qry.InsertTmpDataSql), zap.Error(err)) return err } @@ -179,7 +189,8 @@ func (s *Scope) AlterTableCopy(c *Compile) error { return err } - // 7. drop original table + // 7. drop original table. + // ISCP: That will also drop ISCP related jobs and pitr of the original table. dropSql := fmt.Sprintf("drop table `%s`.`%s`", dbName, tblName) if err := c.runSqlWithOptions( dropSql, @@ -193,26 +204,6 @@ func (s *Scope) AlterTableCopy(c *Compile) error { return err } - // 7.1 delete all index objects of the table in mo_catalog.mo_indexes - if qry.Database != catalog.MO_CATALOG && qry.TableDef.Name != catalog.MO_INDEXES { - if qry.GetTableDef().Pkey != nil || len(qry.GetTableDef().Indexes) > 0 { - deleteSql := fmt.Sprintf( - deleteMoIndexesWithTableIdFormat, - qry.GetTableDef().TblId, - ) - err = c.runSql(deleteSql) - if err != nil { - c.proc.Error(c.proc.Ctx, "delete all index meta data of origin table in `mo_indexes` for alter table", - zap.String("databaseName", dbName), - zap.String("origin tableName", qry.GetTableDef().Name), - zap.String("delete all index sql", deleteSql), - zap.Error(err)) - - return err - } - } - } - newId := newRel.GetTableID(c.proc.Ctx) //------------------------------------------------------------------------- // 8. rename temporary replica table into the original table(Table Id remains unchanged) @@ -237,33 +228,75 @@ func (s *Scope) AlterTableCopy(c *Compile) error { return err } + newTableDef := newRel.CopyTableDef(c.proc.Ctx) //-------------------------------------------------------------------------------------------------------------- { // 9. invoke reindex for the new table, if it contains ivf index. multiTableIndexes := make(map[string]*MultiTableIndex) - newTableDef := newRel.CopyTableDef(c.proc.Ctx) + unaffectedIndexProcessed := make(map[string]bool) extra := newRel.GetExtraInfo() id := newRel.GetTableID(c.proc.Ctx) + isAffectedIndex := func(indexDef *plan.IndexDef, affectedCols []string) bool { + affected := false + for _, part := range indexDef.Parts { + if slices.Index(affectedCols, part) != -1 { + affected = true + break + } + } + return affected + } + for _, indexDef := range newTableDef.Indexes { // DO NOT check SkipIndexesCopy here. SkipIndexesCopy only valids for the unique/master/regular index. // Fulltext/HNSW/Ivfflat indexes are always "unaffected" in skipIndexesCopy // check affectedCols to see it is affected or not. If affected is true, it means the secondary index // are cloned in cloneUnaffectedIndexes(). Otherwise, build the index again. - affected := false - for _, part := range indexDef.Parts { - if slices.Index(qry.AffectedCols, part) != -1 { - affected = true - break + + if !indexDef.Unique && (catalog.IsIvfIndexAlgo(indexDef.IndexAlgo) || + catalog.IsHnswIndexAlgo(indexDef.IndexAlgo) || + catalog.IsFullTextIndexAlgo(indexDef.IndexAlgo)) { + // ivf/hnsw/fulltext index + + if !isAffectedIndex(indexDef, qry.AffectedCols) { + // column not affected means index already cloned in cloneUnaffectedIndexes() + + if unaffectedIndexProcessed[indexDef.IndexName] { + // unaffectedIndex already processed. + continue + } + + valid, err := checkValidIndexCdc(newTableDef, indexDef.IndexName) + if err != nil { + return err + } + + if valid { + // index table may not be fully sync'd with source table via ISCP during alter table + // clone index table (with ISCP) may not be a complete clone + // so register ISCP job with startFromNow = false + sinker_type := getSinkerTypeFromAlgo(indexDef.IndexAlgo) + err = CreateIndexCdcTask(c, dbName, newTableDef.Name, indexDef.IndexName, sinker_type, false) + if err != nil { + return err + } + + logutil.Infof("ISCP register unaffected index db=%s, table=%s, index=%s", dbName, newTableDef.Name, indexDef.IndexName) + } + + unaffectedIndexProcessed[indexDef.IndexName] = true + + continue } - } - if !affected { - // column not affected means index already cloned in cloneUnaffectedIndexes() + } else { + // ignore regular/master/unique index continue } + // only affected ivf/hnsw/fulltext index will go here if catalog.IsIvfIndexAlgo(indexDef.IndexAlgo) || catalog.IsHnswIndexAlgo(indexDef.IndexAlgo) { if _, ok := multiTableIndexes[indexDef.IndexName]; !ok { @@ -272,6 +305,7 @@ func (s *Scope) AlterTableCopy(c *Compile) error { IndexDefs: make(map[string]*plan.IndexDef), } } + ty := catalog.ToLower(indexDef.IndexAlgoTableType) multiTableIndexes[indexDef.IndexName].IndexDefs[ty] = indexDef } @@ -288,6 +322,7 @@ func (s *Scope) AlterTableCopy(c *Compile) error { } } for _, multiTableIndex := range multiTableIndexes { + switch multiTableIndex.IndexAlgo { case catalog.MoIndexIvfFlatAlgo.ToString(): err = s.handleVectorIvfFlatIndex( @@ -603,6 +638,13 @@ func cloneUnaffectedIndexes( AlgoTableType string } + type IndexTableInfo struct { + Unique bool + IndexAlgo string + IndexAlgoParams string + Indexes []IndexTypeInfo + } + var ( clone *table_clone.TableClone @@ -611,8 +653,8 @@ func cloneUnaffectedIndexes( newTblDef = newRel.GetTableDef(c.proc.Ctx) - oriIdxColNameToTblName = make(map[string][]IndexTypeInfo) - newIdxColNameToTblName = make(map[string][]IndexTypeInfo) + oriIdxColNameToTblName = make(map[string]*IndexTableInfo) + newIdxColNameToTblName = make(map[string]*IndexTableInfo) ) logutil.Infof("cloneUnaffectedIndex: affected cols %v\n", affectedCols) @@ -678,10 +720,18 @@ func cloneUnaffectedIndexes( m, ok := oriIdxColNameToTblName[idxTbl.IndexName] if !ok { - m = make([]IndexTypeInfo, 0, 3) + m = &IndexTableInfo{ + Unique: idxTbl.Unique, + IndexAlgo: idxTbl.IndexAlgo, + IndexAlgoParams: idxTbl.IndexAlgoParams, + Indexes: make([]IndexTypeInfo, 0, 3), + } + } - m = append(m, IndexTypeInfo{IndexTableName: idxTbl.IndexTableName, AlgoTableType: idxTbl.IndexAlgoTableType}) + m.Indexes = append(m.Indexes, + IndexTypeInfo{IndexTableName: idxTbl.IndexTableName, + AlgoTableType: idxTbl.IndexAlgoTableType}) oriIdxColNameToTblName[idxTbl.IndexName] = m } @@ -692,10 +742,17 @@ func cloneUnaffectedIndexes( m, ok := newIdxColNameToTblName[idxTbl.IndexName] if !ok { - m = make([]IndexTypeInfo, 0, 3) + m = &IndexTableInfo{ + Unique: idxTbl.Unique, + IndexAlgo: idxTbl.IndexAlgo, + IndexAlgoParams: idxTbl.IndexAlgoParams, + Indexes: make([]IndexTypeInfo, 0, 3), + } } - m = append(m, IndexTypeInfo{IndexTableName: idxTbl.IndexTableName, AlgoTableType: idxTbl.IndexAlgoTableType}) + m.Indexes = append(m.Indexes, + IndexTypeInfo{IndexTableName: idxTbl.IndexTableName, + AlgoTableType: idxTbl.IndexAlgoTableType}) newIdxColNameToTblName[idxTbl.IndexName] = m logutil.Infof("cloneUnaffectedIndex: new %s parts %v\n", idxTbl.IndexTableName, idxTbl.Parts) } @@ -713,11 +770,24 @@ func cloneUnaffectedIndexes( continue } - for _, oriIdxTblName := range oriIdxTblNames { + async, err := catalog.IsIndexAsync(oriIdxTblNames.IndexAlgoParams) + if err != nil { + return err + } + + if !oriIdxTblNames.Unique && + ((catalog.IsFullTextIndexAlgo(oriIdxTblNames.IndexAlgo) && async) || + catalog.IsHnswIndexAlgo(oriIdxTblNames.IndexAlgo)) { + // skip fultext async index and hsnw index clone because index table may not be fully sync'd + logutil.Infof("cloneUnaffectedIndex: skip async index %v\n", oriIdxTblNames) + continue + } + + for _, oriIdxTblName := range oriIdxTblNames.Indexes { var newIdxTblName IndexTypeInfo found := false - for _, idxinfo := range newIdxTblNames { + for _, idxinfo := range newIdxTblNames.Indexes { if oriIdxTblName.AlgoTableType == idxinfo.AlgoTableType { newIdxTblName = idxinfo found = true @@ -729,8 +799,41 @@ func cloneUnaffectedIndexes( continue } + // IVF index table is NOT empty and clone will have duplicate rows + // Delete the table + if !oriIdxTblNames.Unique && + catalog.IsIvfIndexAlgo(oriIdxTblNames.IndexAlgo) { + // delete all content + var sql string + if oriIdxTblName.AlgoTableType == catalog.SystemSI_IVFFLAT_TblType_Metadata { + sql = fmt.Sprintf("DELETE FROM `%s`.`%s` WHERE __mo_index_key = 'version'", dbName, newIdxTblName.IndexTableName) + } else if oriIdxTblName.AlgoTableType == catalog.SystemSI_IVFFLAT_TblType_Centroids { + sql = fmt.Sprintf("DELETE FROM `%s`.`%s` WHERE __mo_index_centroid_version = 0", dbName, newIdxTblName.IndexTableName) + } else if oriIdxTblName.AlgoTableType == catalog.SystemSI_IVFFLAT_TblType_Entries { + sql = fmt.Sprintf("DELETE FROM `%s`.`%s` WHERE __mo_index_centroid_fk_version = 0", dbName, newIdxTblName.IndexTableName) + } else { + return moerr.NewInternalErrorNoCtx(fmt.Sprintf("invalid IVF table type with table %s", newIdxTblName.IndexTableName)) + } + err := c.runSql(sql) + if err != nil { + return err + } + } + + if !oriIdxTblNames.Unique && + async && + catalog.IsIvfIndexAlgo(oriIdxTblNames.IndexAlgo) && + oriIdxTblName.AlgoTableType == catalog.SystemSI_IVFFLAT_TblType_Entries { + // skip async IVF entries index table + logutil.Infof("cloneUnaffectedIndex: skip async IVF entries index table %v\n", oriIdxTblName) + continue + } + logutil.Infof("cloneUnaffectedIndex: clone %v -> %v\n", oriIdxTblName, newIdxTblName) oriIdxObjRef, oriIdxTblDef, err = cctx.Resolve(dbName, oriIdxTblName.IndexTableName, cloneSnapshot) + if err != nil { + return err + } clonePlan := plan.CloneTable{ CreateTable: nil, diff --git a/pkg/sql/compile/ddl.go b/pkg/sql/compile/ddl.go index 0d875baecef32..a7227a575a4ee 100644 --- a/pkg/sql/compile/ddl.go +++ b/pkg/sql/compile/ddl.go @@ -27,6 +27,7 @@ import ( moruntime "github.com/matrixorigin/matrixone/pkg/common/runtime" "github.com/matrixorigin/matrixone/pkg/config" + "github.com/matrixorigin/matrixone/pkg/iscp" "github.com/matrixorigin/matrixone/pkg/pb/task" "github.com/matrixorigin/matrixone/pkg/cdc" @@ -239,6 +240,13 @@ func (s *Scope) DropDatabase(c *Compile) error { return err } } + + // 5.unregister iscp jobs + err = iscp.UnregisterJobsByDBName(c.proc.Ctx, c.proc.GetService(), c.proc.GetTxnOperator(), dbName) + if err != nil { + return err + } + return err } @@ -589,6 +597,13 @@ func (s *Scope) AlterTableInplace(c *Compile) error { newIndexes = append(newIndexes, extra.IndexTables[idx]) } } + + // drop index cdc task + err = DropIndexCdcTask(c, oTableDef, dbName, tblName, constraintName) + if err != nil { + return err + } + // Avoid modifying slice directly during iteration oTableDef.Indexes = notDroppedIndex extra.IndexTables = newIndexes @@ -746,9 +761,8 @@ func (s *Scope) AlterTableInplace(c *Compile) error { if err != nil { return err } - case catalog.MoIndexHnswAlgo.ToString(): - // PASS: keep option unchange for incremental update + // PASS default: return moerr.NewInternalError(c.proc.Ctx, "invalid index algo type for alter reindex") } @@ -890,6 +904,25 @@ func (s *Scope) AlterTableInplace(c *Compile) error { return err } + // post alter table rename -- AlterKind_RenameTable to update iscp job + for _, req := range reqs { + if req.Kind == api.AlterKind_RenameTable { + op, ok := req.Operation.(*api.AlterTableReq_RenameTable) + if ok { + err = iscp.RenameSrcTable(c.proc.Ctx, + c.proc.GetService(), + c.proc.GetTxnOperator(), + req.DbId, + req.TableId, + op.RenameTable.OldName, + op.RenameTable.NewName) + if err != nil { + return err + } + } + } + } + // remove refChildTbls for drop foreign key clause //remove the child table id -- tblId from the parent table -- fkTblId for _, fkTblId := range removeRefChildTbls { @@ -1468,6 +1501,21 @@ func (s *Scope) CreateTable(c *Compile) error { ) return err } + + // create iscp jobs for index async update + ct, err := GetConstraintDef(c.proc.Ctx, newRelation) + if err != nil { + return err + } + for _, constraint := range ct.Cts { + if idxdef, ok := constraint.(*engine.IndexDef); ok && len(idxdef.Indexes) > 0 { + err = CreateAllIndexCdcTasks(c, idxdef.Indexes, dbName, tblName, false) + if err != nil { + return err + } + } + } + } if c.keepAutoIncrement == 0 { @@ -2036,6 +2084,11 @@ func (s *Scope) handleVectorIvfFlatIndex( } } + async, err := catalog.IsIndexAsync(indexDefs[catalog.SystemSI_IVFFLAT_TblType_Metadata].IndexAlgoParams) + if err != nil { + return err + } + // remove the cache with version 0 key := fmt.Sprintf("%s:0", indexDefs[catalog.SystemSI_IVFFLAT_TblType_Centroids].IndexTableName) cache.Cache.Remove(key) @@ -2060,12 +2113,14 @@ func (s *Scope) handleVectorIvfFlatIndex( return err } - // 4.c populate entries table - err = s.handleIvfIndexEntriesTable(c, indexDefs[catalog.SystemSI_IVFFLAT_TblType_Entries], qryDatabase, originalTableDef, - indexDefs[catalog.SystemSI_IVFFLAT_TblType_Metadata].IndexTableName, - indexDefs[catalog.SystemSI_IVFFLAT_TblType_Centroids].IndexTableName) - if err != nil { - return err + if !async { + // 4.c populate entries table + err = s.handleIvfIndexEntriesTable(c, indexDefs[catalog.SystemSI_IVFFLAT_TblType_Entries], qryDatabase, originalTableDef, + indexDefs[catalog.SystemSI_IVFFLAT_TblType_Metadata].IndexTableName, + indexDefs[catalog.SystemSI_IVFFLAT_TblType_Centroids].IndexTableName) + if err != nil { + return err + } } // 4.d delete older entries in index table. @@ -2078,6 +2133,24 @@ func (s *Scope) handleVectorIvfFlatIndex( return err } + /* + // create ISCP job when Async is true + if async { + // unregister ISCP job so that it can restart index update from ts=0 + err = DropIndexCdcTask(c, originalTableDef, qryDatabase, originalTableDef.Name, indexDefs[catalog.SystemSI_IVFFLAT_TblType_Metadata].IndexName) + if err != nil { + return err + } + + logutil.Infof("Ivfflat index Async is true") + sinker_type := getSinkerTypeFromAlgo(catalog.MoIndexIvfFlatAlgo.ToString()) + err = CreateIndexCdcTask(c, qryDatabase, originalTableDef.Name, + indexDefs[catalog.SystemSI_IVFFLAT_TblType_Metadata].IndexName, sinker_type, false) + if err != nil { + return err + } + } + */ return nil } @@ -2102,6 +2175,9 @@ func (s *Scope) DropIndex(c *Compile) error { return err } + // old tabledef + oldTableDef := r.GetTableDef(c.proc.Ctx) + //1. build and update constraint def oldCt, err := GetConstraintDef(c.proc.Ctx, r) if err != nil { @@ -2130,14 +2206,20 @@ func (s *Scope) DropIndex(c *Compile) error { return err } + //3. delete iscp job for vector, fulltext index + err = DropIndexCdcTask(c, oldTableDef, qry.Database, qry.Table, qry.IndexName) + if err != nil { + return err + } } - //3. delete index object from mo_catalog.mo_indexes + //4. delete index object from mo_catalog.mo_indexes deleteSql := fmt.Sprintf(deleteMoIndexesWithTableIdAndIndexNameFormat, r.GetTableID(c.proc.Ctx), qry.IndexName) err = c.runSql(deleteSql) if err != nil { return err } + return nil } @@ -2651,6 +2733,12 @@ func (s *Scope) DropTable(c *Compile) error { } } + // delete cdc task of the vector and fulltext index here + err = DropAllIndexCdcTasks(c, rel.GetTableDef(c.proc.Ctx), qry.Database, qry.Table) + if err != nil { + return err + } + // delete all index objects record of the table in mo_catalog.mo_indexes if !qry.IsView && qry.Database != catalog.MO_CATALOG && qry.Table != catalog.MO_INDEXES { if qry.GetTableDef().Pkey != nil || len(qry.GetTableDef().Indexes) > 0 { diff --git a/pkg/sql/compile/ddl_index_algo.go b/pkg/sql/compile/ddl_index_algo.go index be4eec553a0e0..771541001541e 100644 --- a/pkg/sql/compile/ddl_index_algo.go +++ b/pkg/sql/compile/ddl_index_algo.go @@ -24,6 +24,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/pb/api" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/util/executor" @@ -140,6 +141,7 @@ func (s *Scope) handleFullTextIndexTable( return moerr.NewInternalErrorNoCtx("FullText index is not enabled") } + // create hidden tables if indexInfo != nil { if len(indexInfo.GetIndexTables()) != 1 { return moerr.NewInternalErrorNoCtx("index table count not equal to 1") @@ -152,13 +154,34 @@ func (s *Scope) handleFullTextIndexTable( } } - insertSQLs := genInsertIndexTableSqlForFullTextIndex(originalTableDef, indexDef, qryDatabase) - for _, insertSQL := range insertSQLs { - err = c.runSql(insertSQL) + async, err := catalog.IsIndexAsync(indexDef.IndexAlgoParams) + if err != nil { + return err + } + // create ISCP job for Async fulltext index + if async { + logutil.Infof("fulltext index Async is true") + sinker_type := getSinkerTypeFromAlgo(catalog.MOIndexFullTextAlgo.ToString()) + err = CreateIndexCdcTask(c, qryDatabase, originalTableDef.Name, + indexDef.IndexName, sinker_type, false) if err != nil { return err } + } else { + + insertSQLs, err := genInsertIndexTableSqlForFullTextIndex(originalTableDef, indexDef, qryDatabase) + if err != nil { + return err + } + + for _, insertSQL := range insertSQLs { + err = c.runSql(insertSQL) + if err != nil { + return err + } + } } + return nil } @@ -246,10 +269,11 @@ func (s *Scope) handleIvfIndexCentroidsTable(c *Compile, indexDef *plan.IndexDef return err } + var sql string // 1.b init centroids table with default centroid, if centroids are not enough. // NOTE: we can run re-index to improve the centroid quality. if totalCnt == 0 || totalCnt < int64(centroidParamsLists) { - initSQL := fmt.Sprintf("INSERT INTO `%s`.`%s` (`%s`, `%s`, `%s`) "+ + sql = fmt.Sprintf("INSERT INTO `%s`.`%s` (`%s`, `%s`, `%s`) "+ "SELECT "+ "(SELECT CAST(`%s` AS BIGINT) FROM `%s`.`%s` WHERE `%s` = 'version'), "+ "1, NULL;", @@ -264,60 +288,78 @@ func (s *Scope) handleIvfIndexCentroidsTable(c *Compile, indexDef *plan.IndexDef metadataTableName, catalog.SystemSI_IVFFLAT_TblCol_Metadata_key, ) - err := c.runSql(initSQL) + } else { + + val, err := c.proc.GetResolveVariableFunc()("ivf_threads_build", true, false) if err != nil { return err } - return nil - } + cfg.ThreadsBuild = val.(int64) - val, err := c.proc.GetResolveVariableFunc()("ivf_threads_build", true, false) - if err != nil { - return err - } - cfg.ThreadsBuild = val.(int64) + val, err = c.proc.GetResolveVariableFunc()("kmeans_train_percent", true, false) + if err != nil { + return err + } + cfg.KmeansTrainPercent = val.(int64) - val, err = c.proc.GetResolveVariableFunc()("kmeans_train_percent", true, false) - if err != nil { - return err - } - cfg.KmeansTrainPercent = val.(int64) + val, err = c.proc.GetResolveVariableFunc()("kmeans_max_iteration", true, false) + if err != nil { + return err + } + cfg.KmeansMaxIteration = val.(int64) - val, err = c.proc.GetResolveVariableFunc()("kmeans_max_iteration", true, false) - if err != nil { - return err - } - cfg.KmeansMaxIteration = val.(int64) + params_str := indexDef.IndexAlgoParams - params_str := indexDef.IndexAlgoParams + cfgbytes, err := json.Marshal(cfg) + if err != nil { + return err + } - cfgbytes, err := json.Marshal(cfg) - if err != nil { - return err + part := src_alias + "." + indexDef.Parts[0] + insertIntoIvfIndexTableFormat := "SELECT f.* from `%s`.`%s` AS %s CROSS APPLY ivf_create('%s', '%s', %s) AS f;" + sql = fmt.Sprintf(insertIntoIvfIndexTableFormat, + qryDatabase, originalTableDef.Name, + src_alias, + params_str, + string(cfgbytes), + part) } - part := src_alias + "." + indexDef.Parts[0] - insertIntoIvfIndexTableFormat := "SELECT f.* from `%s`.`%s` AS %s CROSS APPLY ivf_create('%s', '%s', %s) AS f;" - sql := fmt.Sprintf(insertIntoIvfIndexTableFormat, - qryDatabase, originalTableDef.Name, - src_alias, - params_str, - string(cfgbytes), - part) - - err = s.logTimestamp(c, qryDatabase, metadataTableName, "clustering_start") + async, err := catalog.IsIndexAsync(indexDef.IndexAlgoParams) if err != nil { return err } + if async { - err = c.runSql(sql) - if err != nil { - return err - } + // create ISCP job when Async is true + // unregister ISCP job so that it can restart index update from ts=0 + err = DropIndexCdcTask(c, originalTableDef, qryDatabase, originalTableDef.Name, indexDef.IndexName) + if err != nil { + return err + } - err = s.logTimestamp(c, qryDatabase, metadataTableName, "clustering_end") - if err != nil { - return err + logutil.Infof("Ivfflat index Async is true") + sinker_type := getSinkerTypeFromAlgo(catalog.MoIndexIvfFlatAlgo.ToString()) + err = AppendIscpRegisterEvent(c, qryDatabase, originalTableDef.Name, indexDef.IndexName, sinker_type, false, sql) + if err != nil { + return err + } + + } else { + err = s.logTimestamp(c, qryDatabase, metadataTableName, "clustering_start") + if err != nil { + return err + } + + err = c.runSql(sql) + if err != nil { + return err + } + + err = s.logTimestamp(c, qryDatabase, metadataTableName, "clustering_end") + if err != nil { + return err + } } return nil @@ -567,14 +609,44 @@ func (s *Scope) handleVectorHnswIndex( } } - // 3. build hnsw index - sqls, err := genBuildHnswIndex(c.proc, indexDefs, qryDatabase, originalTableDef) + async, err := catalog.IsIndexAsync(indexDefs[catalog.Hnsw_TblType_Metadata].IndexAlgoParams) if err != nil { return err } - for _, sql := range sqls { - err = c.runSql(sql) + if !async { + // 3. build hnsw index + sqls, err := genBuildHnswIndex(c.proc, indexDefs, qryDatabase, originalTableDef) + if err != nil { + return err + } + + for _, sql := range sqls { + err = c.runSql(sql) + if err != nil { + return err + } + } + + // register ISCP job with startFromNow = true + // 4. register ISCP job for async update + sinker_type := getSinkerTypeFromAlgo(catalog.MoIndexHnswAlgo.ToString()) + err = CreateIndexCdcTask(c, qryDatabase, originalTableDef.Name, indexDefs[catalog.Hnsw_TblType_Metadata].IndexName, sinker_type, true) + if err != nil { + return err + } + } + + if async { + // unregister ISCP job + err = DropIndexCdcTask(c, originalTableDef, qryDatabase, originalTableDef.Name, indexDefs[catalog.Hnsw_TblType_Metadata].IndexName) + if err != nil { + return err + } + + // 4. register ISCP job for async update with startFromNow = false + sinker_type := getSinkerTypeFromAlgo(catalog.MoIndexHnswAlgo.ToString()) + err := CreateIndexCdcTask(c, qryDatabase, originalTableDef.Name, indexDefs[catalog.Hnsw_TblType_Metadata].IndexName, sinker_type, false) if err != nil { return err } diff --git a/pkg/sql/compile/iscp_util.go b/pkg/sql/compile/iscp_util.go new file mode 100644 index 0000000000000..5b6c0efd0742a --- /dev/null +++ b/pkg/sql/compile/iscp_util.go @@ -0,0 +1,321 @@ +// Copyright 2023 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package compile + +import ( + "context" + "time" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/defines" + "github.com/matrixorigin/matrixone/pkg/iscp" + "github.com/matrixorigin/matrixone/pkg/logutil" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/pb/txn" + "github.com/matrixorigin/matrixone/pkg/txn/client" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" + "github.com/matrixorigin/matrixone/pkg/vm/engine" +) + +var ( + iscpRegisterJobFunc = iscp.RegisterJob + iscpUnregisterJobFunc = iscp.UnregisterJob +) + +/* CDC APIs */ +func RegisterJob(ctx context.Context, cnUUID string, txn client.TxnOperator, spec *iscp.JobSpec, job *iscp.JobID, startFromNow bool) (bool, error) { + //dummyurl := "mysql://root:111@127.0.0.1:6001" + // sql = fmt.Sprintf("CREATE CDC `%s` '%s' 'indexsync' '%s' '%s.%s' {'Level'='table'};", cdcname, dummyurl, dummyurl, qryDatabase, srctbl) + return iscpRegisterJobFunc(ctx, cnUUID, txn, spec, job, startFromNow) +} + +func UnregisterJob(ctx context.Context, cnUUID string, txn client.TxnOperator, job *iscp.JobID) (bool, error) { + return iscpUnregisterJobFunc(ctx, cnUUID, txn, job) +} + +/* start here */ +func CreateCdcTask(c *Compile, spec *iscp.JobSpec, job *iscp.JobID, startFromNow bool) (bool, error) { + logutil.Infof("Create Index Task %v", spec) + + return RegisterJob(c.proc.Ctx, c.proc.GetService(), c.proc.GetTxnOperator(), spec, job, startFromNow) +} + +func DeleteCdcTask(c *Compile, job *iscp.JobID) (bool, error) { + logutil.Infof("Delete Index Task %v", job) + return UnregisterJob(c.proc.Ctx, c.proc.GetService(), c.proc.GetTxnOperator(), job) +} + +func checkValidIndexCdcByIndexdef(idx *plan.IndexDef) (bool, error) { + var err error + + if idx.TableExist && + (catalog.IsHnswIndexAlgo(idx.IndexAlgo) || + catalog.IsIvfIndexAlgo(idx.IndexAlgo) || + catalog.IsFullTextIndexAlgo(idx.IndexAlgo)) { + async := false + if catalog.IsHnswIndexAlgo(idx.IndexAlgo) { + // HNSW always async + async = true + } else { + async, err = catalog.IsIndexAsync(idx.IndexAlgoParams) + if err != nil { + return false, err + } + } + + return async, nil + } + return false, nil +} + +func checkValidIndexCdc(tableDef *plan.TableDef, indexname string) (bool, error) { + for _, idx := range tableDef.Indexes { + + if idx.IndexName == indexname { + valid, err := checkValidIndexCdcByIndexdef(idx) + if err != nil { + return false, err + } + if valid { + return true, nil + } + } + } + return false, nil +} + +// NOTE: CreateIndexCdcTask will create CDC task without any checking. Original TableDef may be empty +func CreateIndexCdcTask(c *Compile, dbname string, tablename string, indexname string, sinker_type int8, startFromNow bool) error { + var err error + + spec := &iscp.JobSpec{ + ConsumerInfo: iscp.ConsumerInfo{ConsumerType: sinker_type, + DBName: dbname, + TableName: tablename, + IndexName: indexname}, + } + job := &iscp.JobID{DBName: dbname, TableName: tablename, JobName: genCdcTaskJobID(indexname)} + + // create index cdc task + ok, err := CreateCdcTask(c, spec, job, startFromNow) + if err != nil { + return err + } + + if !ok { + // cdc task already exist. ignore it. IVFFLAT alter reindex will call CreateIndexCdcTask multiple times. + logutil.Infof("index cdc task (%s, %s, %s) already exists", dbname, tablename, indexname) + return nil + } + return nil +} + +func genCdcTaskJobID(indexname string) string { + return "index_" + indexname +} + +func DropIndexCdcTask(c *Compile, tableDef *plan.TableDef, dbname string, tablename string, indexname string) error { + var err error + + valid, err := checkValidIndexCdc(tableDef, indexname) + if err != nil { + return err + } + + if !valid { + // index name is not valid cdc task. ignore it + return nil + } + + // delete index cdc task + _, err = DeleteCdcTask(c, &iscp.JobID{DBName: dbname, TableName: tablename, JobName: genCdcTaskJobID(indexname)}) + if err != nil { + return err + } + + return nil +} + +// drop all cdc tasks according to tableDef +func DropAllIndexCdcTasks(c *Compile, tabledef *plan.TableDef, dbname string, tablename string) error { + idxmap := make(map[string]bool) + for _, idx := range tabledef.Indexes { + + _, ok := idxmap[idx.IndexName] + if ok { + continue + } + + valid, err := checkValidIndexCdcByIndexdef(idx) + if err != nil { + return err + } + + if valid { + idxmap[idx.IndexName] = true + //hasindex = true + _, e := DeleteCdcTask(c, &iscp.JobID{DBName: dbname, TableName: tablename, JobName: genCdcTaskJobID(idx.IndexName)}) + if e != nil { + return e + } + } + } + return nil +} + +func getSinkerTypeFromAlgo(algo string) int8 { + if catalog.IsHnswIndexAlgo(algo) { + return int8(iscp.ConsumerType_IndexSync) + } else if catalog.IsIvfIndexAlgo(algo) { + return int8(iscp.ConsumerType_IndexSync) + } else if catalog.IsFullTextIndexAlgo(algo) { + return int8(iscp.ConsumerType_IndexSync) + } + panic("getSinkerTypeFromAlgo: invalid sinker type") +} + +// NOTE: CreateAllIndexCdcTasks will create CDC task according to existing tableDef +func CreateAllIndexCdcTasks(c *Compile, indexes []*plan.IndexDef, dbname string, tablename string, startFromNow bool) error { + idxmap := make(map[string]bool) + for _, idx := range indexes { + _, ok := idxmap[idx.IndexName] + if ok { + continue + } + + valid, err := checkValidIndexCdcByIndexdef(idx) + if err != nil { + return err + } + + if valid { + idxmap[idx.IndexName] = true + sinker_type := getSinkerTypeFromAlgo(idx.IndexAlgo) + e := CreateIndexCdcTask(c, dbname, tablename, idx.IndexName, sinker_type, startFromNow) + if e != nil { + return e + } + } + } + return nil +} + +type TxnCallbackData struct { + cnUUID string + txnClient client.TxnClient + cnEngine engine.Engine + accountId uint32 + dbname string + tablename string + indexname string + sinker_type int8 + startFromNow bool + resolveVariableFunc func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error) + sql string +} + +func iscpRegisterEventCallbackFn(sqlproc *sqlexec.SqlProcess, data any) (err error) { + sqlctx := sqlproc.SqlCtx + cbdata := data.(TxnCallbackData) + + // if sql is not empty, execute the SQL + if len(cbdata.sql) > 0 { + res, err := sqlexec.RunSql(sqlproc, cbdata.sql) + if err != nil { + return err + } + res.Close() + } + + // register ISCP job + spec := &iscp.JobSpec{ + ConsumerInfo: iscp.ConsumerInfo{ConsumerType: cbdata.sinker_type, + DBName: cbdata.dbname, + TableName: cbdata.tablename, + IndexName: cbdata.indexname}, + } + job := &iscp.JobID{DBName: cbdata.dbname, TableName: cbdata.tablename, JobName: genCdcTaskJobID(cbdata.indexname)} + + _, err = RegisterJob(sqlproc.GetContext(), sqlctx.GetService(), sqlctx.Txn(), spec, job, cbdata.startFromNow) + if err != nil { + return + } + return +} + +func AppendIscpRegisterEvent(c *Compile, dbname string, tablename string, indexname string, sinker_type int8, startFromNow bool, sql string) error { + + accountId, err := defines.GetAccountId(c.proc.Ctx) + if err != nil { + return err + } + + cbdata := TxnCallbackData{cnUUID: c.proc.GetService(), + txnClient: c.proc.Base.TxnClient, + cnEngine: c.proc.Base.SessionInfo.StorageEngine, + accountId: accountId, + dbname: dbname, + tablename: tablename, + indexname: indexname, + sinker_type: sinker_type, + startFromNow: startFromNow, + resolveVariableFunc: c.proc.GetResolveVariableFunc(), + sql: sql, + } + txnop := c.proc.GetTxnOperator() + txnop.AppendEventCallback(client.ClosedEvent, + client.NewTxnEventCallbackWithValue( + func(ctx context.Context, _ client.TxnOperator, evt client.TxnEvent, data any) (err error) { + //commitTs := evt.Txn.CommitTS + + cbdata := data.(TxnCallbackData) + logutil.Infof("AppendIscpRegisterEvent: closed event detected\n %v", cbdata) + if evt.Txn.Status != txn.TxnStatus_Committed { + return + } + + if len(cbdata.sql) > 0 { + // long running SQL so run in separate thread + go func() { + sqlexec.RunTxnWithSqlContext(ctx, + cbdata.cnEngine, + cbdata.txnClient, + cbdata.cnUUID, + cbdata.accountId, + 24*time.Hour, + cbdata.resolveVariableFunc, + cbdata, + iscpRegisterEventCallbackFn) + + }() + return + + } else { + return sqlexec.RunTxnWithSqlContext(ctx, + cbdata.cnEngine, + cbdata.txnClient, + cbdata.cnUUID, + cbdata.accountId, + 5*time.Minute, + cbdata.resolveVariableFunc, + cbdata, + iscpRegisterEventCallbackFn) + } + + }, cbdata)) + + return nil + +} diff --git a/pkg/sql/compile/iscp_util_test.go b/pkg/sql/compile/iscp_util_test.go new file mode 100644 index 0000000000000..05d4fa75595bc --- /dev/null +++ b/pkg/sql/compile/iscp_util_test.go @@ -0,0 +1,301 @@ +// Copyright 2023 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package compile + +import ( + "context" + "fmt" + "testing" + + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/iscp" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/testutil" + "github.com/matrixorigin/matrixone/pkg/txn/client" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +/* +func mockIscpRegisterJobSuccess(ctx context.Context, cnUUID string, txn client.TxnOperator, spec *iscp.JobSpec, job *iscp.JobID, startFromNow bool) (bool, error) { + return true, nil +} + +func mockIscpUnregisterJobSuccess(ctx context.Context, cnUUID string, txn client.TxnOperator, job *iscp.JobID) (bool, error) { + return true, nil +} +*/ + +func mockIscpRegisterJobError(ctx context.Context, cnUUID string, txn client.TxnOperator, spec *iscp.JobSpec, job *iscp.JobID, startFromNow bool) (bool, error) { + return false, moerr.NewInternalErrorNoCtx("mock register job error") +} + +func mockIscpUnregisterJobError(ctx context.Context, cnUUID string, txn client.TxnOperator, job *iscp.JobID) (bool, error) { + return false, moerr.NewInternalErrorNoCtx("mock unregister job error") +} + +func TestISCPCheckValidIndexCdcByIndexdef(t *testing.T) { + { + + idx := &plan.IndexDef{ + TableExist: true, + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":"true"}`, + } + found, err := checkValidIndexCdcByIndexdef(idx) + require.Nil(t, err) + require.Equal(t, found, true) + } + + { + + idx := &plan.IndexDef{ + TableExist: true, + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":1}`, + } + _, err := checkValidIndexCdcByIndexdef(idx) + require.NotNil(t, err) + } + + { + + idx := &plan.IndexDef{ + TableExist: true, + IndexAlgo: "ivfflat", + IndexAlgoParams: `{}`, + } + found, err := checkValidIndexCdcByIndexdef(idx) + require.Nil(t, err) + require.Equal(t, found, false) + } +} + +func TestISCPCheckValidIndexCdc(t *testing.T) { + + { + tbldef := &plan.TableDef{ + Indexes: []*plan.IndexDef{ + { + TableExist: true, + IndexName: "a", + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":"true"}`, + }, + }, + } + + ok, err := checkValidIndexCdc(tbldef, "a") + require.Nil(t, err) + require.Equal(t, ok, true) + + } + + { + tbldef := &plan.TableDef{ + Indexes: []*plan.IndexDef{ + { + TableExist: true, + IndexName: "a", + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":1}`, + }, + }, + } + + _, err := checkValidIndexCdc(tbldef, "a") + require.NotNil(t, err) + + } +} + +func TestISCPCreateAllIndexCdcTasks(t *testing.T) { + + iscpRegisterJobFunc = mockIscpRegisterJobError + iscpUnregisterJobFunc = mockIscpUnregisterJobError + + defer func() { + iscpRegisterJobFunc = iscp.RegisterJob + iscpUnregisterJobFunc = iscp.UnregisterJob + }() + + c := &Compile{} + c.proc = testutil.NewProcess(t) + + { + tbldef := &plan.TableDef{ + Indexes: []*plan.IndexDef{ + { + TableExist: true, + IndexName: "a", + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":"true"}`, + }, + }, + } + + err := CreateAllIndexCdcTasks(c, tbldef.Indexes, "dbname", "tname", false) + require.NotNil(t, err) + fmt.Println(err) + + } + + { + tbldef := &plan.TableDef{ + Indexes: []*plan.IndexDef{ + { + TableExist: true, + IndexName: "a", + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":1}`, + }, + }, + } + + err := CreateAllIndexCdcTasks(c, tbldef.Indexes, "dbname", "tname", false) + require.NotNil(t, err) + fmt.Println(err) + + } + +} + +func TestISCPDropAllIndexCdcTasks(t *testing.T) { + + iscpRegisterJobFunc = mockIscpRegisterJobError + iscpUnregisterJobFunc = mockIscpUnregisterJobError + + defer func() { + iscpRegisterJobFunc = iscp.RegisterJob + iscpUnregisterJobFunc = iscp.UnregisterJob + }() + + c := &Compile{} + c.proc = testutil.NewProcess(t) + + { + tbldef := &plan.TableDef{ + Indexes: []*plan.IndexDef{ + { + TableExist: true, + IndexName: "a", + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":"true"}`, + }, + }, + } + + err := DropAllIndexCdcTasks(c, tbldef, "dbname", "tname") + require.NotNil(t, err) + fmt.Println(err) + + } + + { + tbldef := &plan.TableDef{ + Indexes: []*plan.IndexDef{ + { + TableExist: true, + IndexName: "a", + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":1}`, + }, + }, + } + + err := DropAllIndexCdcTasks(c, tbldef, "dbname", "tname") + require.NotNil(t, err) + fmt.Println(err) + + } + +} + +func TestISCPDropIndexCdcTask(t *testing.T) { + + iscpRegisterJobFunc = mockIscpRegisterJobError + iscpUnregisterJobFunc = mockIscpUnregisterJobError + + defer func() { + iscpRegisterJobFunc = iscp.RegisterJob + iscpUnregisterJobFunc = iscp.UnregisterJob + }() + + c := &Compile{} + c.proc = testutil.NewProcess(t) + + { + tbldef := &plan.TableDef{ + Indexes: []*plan.IndexDef{ + { + TableExist: true, + IndexName: "a", + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":"true"}`, + }, + }, + } + + err := DropIndexCdcTask(c, tbldef, "dbname", "tname", "a") + require.NotNil(t, err) + fmt.Println(err) + + } + + { + tbldef := &plan.TableDef{ + Indexes: []*plan.IndexDef{ + { + TableExist: true, + IndexName: "a", + IndexAlgo: "ivfflat", + IndexAlgoParams: `{"async":1}`, + }, + }, + } + + err := DropIndexCdcTask(c, tbldef, "dbname", "tname", "a") + require.NotNil(t, err) + fmt.Println(err) + + } + +} + +func TestISCPCreateIndexCdcTask(t *testing.T) { + + iscpRegisterJobFunc = mockIscpRegisterJobError + iscpUnregisterJobFunc = mockIscpUnregisterJobError + + defer func() { + iscpRegisterJobFunc = iscp.RegisterJob + iscpUnregisterJobFunc = iscp.UnregisterJob + }() + + c := &Compile{} + c.proc = testutil.NewProcess(t) + + { + err := CreateIndexCdcTask(c, "dbname", "tname", "a", 0, true) + require.NotNil(t, err) + fmt.Println(err) + + } + +} + +func TestISCPGetSinkerTypeFromAlgo(t *testing.T) { + assert.Panics(t, func() { getSinkerTypeFromAlgo("error") }, "getSinkerTypeFromAlgo panic") +} diff --git a/pkg/sql/compile/util.go b/pkg/sql/compile/util.go index 8fa97ba42d952..ac11b96982751 100644 --- a/pkg/sql/compile/util.go +++ b/pkg/sql/compile/util.go @@ -505,7 +505,7 @@ func GetConstraintDefFromTableDefs(defs []engine.TableDef) *engine.ConstraintDef return cstrDef } -func genInsertIndexTableSqlForFullTextIndex(originalTableDef *plan.TableDef, indexDef *plan.IndexDef, qryDatabase string) []string { +func genInsertIndexTableSqlForFullTextIndex(originalTableDef *plan.TableDef, indexDef *plan.IndexDef, qryDatabase string) ([]string, error) { src_alias := "src" pkColName := src_alias + "." + originalTableDef.Pkey.PkeyColName params := indexDef.IndexAlgoParams @@ -526,7 +526,7 @@ func genInsertIndexTableSqlForFullTextIndex(originalTableDef *plan.TableDef, ind pkColName, concat) - return []string{sql} + return []string{sql}, nil } func genDeleteHnswIndex(proc *process.Process, indexDefs map[string]*plan.IndexDef, qryDatabase string, originalTableDef *plan.TableDef) ([]string, error) { diff --git a/pkg/sql/plan/build_dml_util.go b/pkg/sql/plan/build_dml_util.go index 63186d2c7c8b1..a434b9f68e222 100644 --- a/pkg/sql/plan/build_dml_util.go +++ b/pkg/sql/plan/build_dml_util.go @@ -299,18 +299,18 @@ func checkDeleteOptToTruncate(ctx CompilerContext) (bool, error) { // buildDeletePlans build preinsert plan. /* -[o1]sink_scan -> join[u1] -> sink - [u1]sink_scan -> lock -> delete -> [mergedelete] ... // if it's delete stmt. do delete u1 - [u1]sink_scan -> preinsert_uk -> sink ... // if it's update stmt. do update u1 -[o1]sink_scan -> join[u2] -> sink - [u2]sink_scan -> lock -> delete -> [mergedelete] ... // if it's delete stmt. do delete u2 - [u2]sink_scan -> preinsert_uk -> sink ... // if it's update stmt. do update u2 -[o1]sink_scan -> predelete[get partition] -> lock -> delete -> [mergedelete] - -[o1]sink_scan -> join[f1 semi join c1 on c1.fid=f1.id, get f1.id] -> filter(assert(isempty(id))) // if have refChild table with no action -[o1]sink_scan -> join[f1 inner join c2 on f1.id = c2.fid, 取c2.*, null] -> sink ...(like update) // if have refChild table with set null -[o1]sink_scan -> join[f1 inner join c4 on f1.id = c4.fid, get c3.*] -> sink ...(like delete) // delete stmt: if have refChild table with cascade -[o1]sink_scan -> join[f1 inner join c4 on f1.id = c4.fid, get c3.*, update cols] -> sink ...(like update) // update stmt: if have refChild table with cascade + [o1]sink_scan -> join[u1] -> sink + [u1]sink_scan -> lock -> delete -> [mergedelete] ... // if it's delete stmt. do delete u1 + [u1]sink_scan -> preinsert_uk -> sink ... // if it's update stmt. do update u1 + [o1]sink_scan -> join[u2] -> sink + [u2]sink_scan -> lock -> delete -> [mergedelete] ... // if it's delete stmt. do delete u2 + [u2]sink_scan -> preinsert_uk -> sink ... // if it's update stmt. do update u2 + [o1]sink_scan -> predelete[get partition] -> lock -> delete -> [mergedelete] + + [o1]sink_scan -> join[f1 semi join c1 on c1.fid=f1.id, get f1.id] -> filter(assert(isempty(id))) // if have refChild table with no action + [o1]sink_scan -> join[f1 inner join c2 on f1.id = c2.fid, 取c2.*, null] -> sink ...(like update) // if have refChild table with set null + [o1]sink_scan -> join[f1 inner join c4 on f1.id = c4.fid, get c3.*] -> sink ...(like delete) // delete stmt: if have refChild table with cascade + [o1]sink_scan -> join[f1 inner join c4 on f1.id = c4.fid, get c3.*, update cols] -> sink ...(like update) // update stmt: if have refChild table with cascade */ func buildDeletePlans(ctx CompilerContext, builder *QueryBuilder, bindCtx *BindContext, delCtx *dmlPlanCtx) error { if sinkOrUnionNodeId, ok := builder.deleteNode[delCtx.tableDef.TblId]; ok { @@ -3202,16 +3202,16 @@ func runSql(ctx CompilerContext, sql string) (executor.Result, error) { } /* -Example on FkReferKey and FkReferDef: + Example on FkReferKey and FkReferDef: - In database `test`: + In database `test`: - create table t1(a int,primary key(a)); + create table t1(a int,primary key(a)); - create table t2(b int, constraint c1 foreign key(b) references t1(a)); + create table t2(b int, constraint c1 foreign key(b) references t1(a)); - So, the structure FkReferDef below denotes such relationships : test.t2(b) -> test.t1(a) - FkReferKey holds : db = test, tbl = t2 + So, the structure FkReferDef below denotes such relationships : test.t2(b) -> test.t1(a) + FkReferKey holds : db = test, tbl = t2 */ @@ -3507,6 +3507,16 @@ func buildPreInsertMultiTableIndexes(ctx CompilerContext, builder *QueryBuilder, switch multiTableIndex.IndexAlgo { case catalog.MoIndexIvfFlatAlgo.ToString(): + // skip async + var async bool + async, err = catalog.IsIndexAsync(multiTableIndex.IndexAlgoParams) + if err != nil { + return err + } + if async { + continue + } + lastNodeId = appendSinkScanNode(builder, bindCtx, sourceStep) var idxRefs = make([]*ObjectRef, 3) var idxTableDefs = make([]*TableDef, 3) @@ -3580,6 +3590,16 @@ func buildDeleteMultiTableIndexes(ctx CompilerContext, builder *QueryBuilder, bi for _, multiTableIndex := range multiTableIndexes { switch multiTableIndex.IndexAlgo { case catalog.MoIndexIvfFlatAlgo.ToString(): + // skip async + var async bool + async, err = catalog.IsIndexAsync(multiTableIndex.IndexAlgoParams) + if err != nil { + return err + } + if async { + continue + } + // Used by pre-insert vector index. var idxRefs = make([]*ObjectRef, 3) var idxTableDefs = make([]*TableDef, 3) @@ -4297,6 +4317,15 @@ func buildDeleteIndexPlans(ctx CompilerContext, builder *QueryBuilder, bindCtx * func buildPreInsertFullTextIndex(stmt *tree.Insert, ctx CompilerContext, builder *QueryBuilder, bindCtx *BindContext, objRef *ObjectRef, tableDef *TableDef, updateColLength int, sourceStep int32, ifInsertFromUniqueColMap map[string]bool, indexdef *plan.IndexDef, idx int) error { + // skip async + async, err := catalog.IsIndexAsync(indexdef.IndexAlgoParams) + if err != nil { + return err + } + if async { + return nil + } + isUpdate := (updateColLength > 0) lastNodeId := appendSinkScanNode(builder, bindCtx, sourceStep) @@ -4718,6 +4747,15 @@ func buildDeleteRowsFullTextIndex(ctx CompilerContext, builder *QueryBuilder, bi func buildPreDeleteFullTextIndex(ctx CompilerContext, builder *QueryBuilder, bindCtx *BindContext, delCtx *dmlPlanCtx, indexdef *plan.IndexDef, idx int, typMap map[string]plan.Type, posMap map[string]int) error { + // skip async + async, err := catalog.IsIndexAsync(indexdef.IndexAlgoParams) + if err != nil { + return err + } + if async { + return nil + } + //isUpdate := delCtx.updateColLength > 0 indexObjRef, indexTableDef, err := ctx.ResolveIndexTableByRef(delCtx.objRef, indexdef.IndexTableName, nil) if err != nil { @@ -4749,6 +4787,15 @@ func buildPreDeleteFullTextIndex(ctx CompilerContext, builder *QueryBuilder, bin func buildPostDmlFullTextIndex(ctx CompilerContext, builder *QueryBuilder, bindCtx *BindContext, indexObjRef *ObjectRef, indexTableDef *TableDef, tableDef *TableDef, sourceStep int32, indexdef *plan.IndexDef, idx int, isDelete, isInsert, isDeleteWithoutFilters bool) error { + // skip async + async, err := catalog.IsIndexAsync(indexdef.IndexAlgoParams) + if err != nil { + return err + } + if async { + return nil + } + lastNodeId := appendSinkScanNode(builder, bindCtx, sourceStep) orgPkColPos, _ := getPkPos(tableDef, false) diff --git a/pkg/sql/plan/build_dml_util_test.go b/pkg/sql/plan/build_dml_util_test.go index 7008a31d87d60..2fe741e4545c5 100644 --- a/pkg/sql/plan/build_dml_util_test.go +++ b/pkg/sql/plan/build_dml_util_test.go @@ -23,6 +23,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/buffer" moruntime "github.com/matrixorigin/matrixone/pkg/common/runtime" + "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/testutil" "github.com/matrixorigin/matrixone/pkg/util/executor" ) @@ -50,3 +51,51 @@ func Test_runSql(t *testing.T) { _, err := runSql(compilerContext, "") require.Error(t, err, "internal error: no account id in context") } + +func Test_buildPostDmlFullTextIndexAsync(t *testing.T) { + { + //invalid json + idxdef := &plan.IndexDef{ + IndexAlgoParams: `{"async":1}`, + } + + err := buildPostDmlFullTextIndex(nil, nil, nil, nil, nil, nil, 0, idxdef, 0, false, false, false) + require.NotNil(t, err) + } + + { + + // async true + idxdef := &plan.IndexDef{ + IndexAlgoParams: `{"async":"true"}`, + } + + err := buildPostDmlFullTextIndex(nil, nil, nil, nil, nil, nil, 0, idxdef, 0, false, false, false) + require.Nil(t, err) + } + +} + +func Test_buildPreDeleteFullTextIndexAsync(t *testing.T) { + { + //invalid json + idxdef := &plan.IndexDef{ + IndexAlgoParams: `{"async":1}`, + } + + err := buildPreDeleteFullTextIndex(nil, nil, nil, nil, idxdef, 0, nil, nil) + require.NotNil(t, err) + } + + { + + // async true + idxdef := &plan.IndexDef{ + IndexAlgoParams: `{"async":"true"}`, + } + + err := buildPreDeleteFullTextIndex(nil, nil, nil, nil, idxdef, 0, nil, nil) + require.Nil(t, err) + } + +} diff --git a/pkg/sql/plan/function/func_hnsw.go b/pkg/sql/plan/function/func_hnsw.go deleted file mode 100644 index 07b20235a9ba8..0000000000000 --- a/pkg/sql/plan/function/func_hnsw.go +++ /dev/null @@ -1,106 +0,0 @@ -// Copyright 2021 - 2022 Matrix Origin -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package function - -import ( - "encoding/json" - - "github.com/matrixorigin/matrixone/pkg/common/moerr" - "github.com/matrixorigin/matrixone/pkg/container/types" - "github.com/matrixorigin/matrixone/pkg/container/vector" - "github.com/matrixorigin/matrixone/pkg/logutil" - "github.com/matrixorigin/matrixone/pkg/vectorindex" - "github.com/matrixorigin/matrixone/pkg/vectorindex/hnsw" - "github.com/matrixorigin/matrixone/pkg/vm/process" -) - -func hnswCdcUpdate(ivecs []*vector.Vector, result vector.FunctionResultWrapper, proc *process.Process, length int, selectList *FunctionSelectList) error { - - if len(ivecs) != 5 { - return moerr.NewInvalidInput(proc.Ctx, "number of arguments != 5") - } - - dbVec := vector.GenerateFunctionStrParameter(ivecs[0]) - tblVec := vector.GenerateFunctionStrParameter(ivecs[1]) - typeVec := vector.GenerateFunctionFixedTypeParameter[int32](ivecs[2]) - dimVec := vector.GenerateFunctionFixedTypeParameter[int32](ivecs[3]) - cdcVec := vector.GenerateFunctionStrParameter(ivecs[4]) - - for i := uint64(0); i < uint64(length); i++ { - dbname, isnull := dbVec.GetStrValue(i) - if isnull { - return moerr.NewInvalidInput(proc.Ctx, "dbname is null") - } - - tblname, isnull := tblVec.GetStrValue(i) - if isnull { - return moerr.NewInvalidInput(proc.Ctx, "table name is null") - - } - - typ, isnull := typeVec.GetValue(i) - if isnull { - return moerr.NewInvalidInput(proc.Ctx, "type is null") - } - - dim, isnull := dimVec.GetValue(i) - if isnull { - return moerr.NewInvalidInput(proc.Ctx, "dimension is null") - } - - cdcstr, isnull := cdcVec.GetStrValue(i) - if isnull { - return moerr.NewInvalidInput(proc.Ctx, "cdc is null") - } - - switch typ { - case int32(types.T_array_float32): - var cdc vectorindex.VectorIndexCdc[float32] - err := json.Unmarshal([]byte(cdcstr), &cdc) - if err != nil { - return moerr.NewInvalidInput(proc.Ctx, "cdc is not json object") - } - - logutil.Infof("hnsw_cdc_update: START db=%s, table=%s\n", dbname, tblname) - // hnsw sync - //os.Stderr.WriteString(fmt.Sprintf("db=%s, table=%s, dim=%d, json=%s\n", dbname, tblname, dim, cdcstr)) - err = hnsw.CdcSync[float32](proc, string(dbname), string(tblname), typ, dim, &cdc) - if err != nil { - return err - } - logutil.Infof("hnsw_cdc_update: END db=%s, table=%s\n", dbname, tblname) - case int32(types.T_array_float64): - var cdc vectorindex.VectorIndexCdc[float64] - err := json.Unmarshal([]byte(cdcstr), &cdc) - if err != nil { - return moerr.NewInvalidInput(proc.Ctx, "cdc is not json object") - } - - logutil.Infof("hnsw_cdc_update: START db=%s, table=%s\n", dbname, tblname) - // hnsw sync - //os.Stderr.WriteString(fmt.Sprintf("db=%s, table=%s, dim=%d, json=%s\n", dbname, tblname, dim, cdcstr)) - err = hnsw.CdcSync[float64](proc, string(dbname), string(tblname), typ, dim, &cdc) - if err != nil { - return err - } - logutil.Infof("hnsw_cdc_update: END db=%s, table=%s\n", dbname, tblname) - default: - return moerr.NewInvalidInput(proc.Ctx, "invalid vector type") - - } - } - - return nil -} diff --git a/pkg/sql/plan/function/func_hnsw_test.go b/pkg/sql/plan/function/func_hnsw_test.go deleted file mode 100644 index e8306fa894fc6..0000000000000 --- a/pkg/sql/plan/function/func_hnsw_test.go +++ /dev/null @@ -1,191 +0,0 @@ -// Copyright 2021 - 2022 Matrix Origin -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package function - -import ( - "testing" - - "github.com/matrixorigin/matrixone/pkg/container/types" - "github.com/matrixorigin/matrixone/pkg/testutil" - "github.com/stretchr/testify/require" -) - -func TestHnswCdcUpdateFn(t *testing.T) { - tcs := []tcTemp{ - { - info: "nargs invalid", - inputs: []FunctionTestInput{ - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{true}), - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - }, - expect: NewFunctionTestResult(types.T_int64.ToType(), true, - []int64{0}, []bool{false}), - }, - - { - info: "dbname null", - inputs: []FunctionTestInput{ - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{true}), - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_json.ToType(), - []string{""}, []bool{false}), - }, - expect: NewFunctionTestResult(types.T_int64.ToType(), true, - []int64{0}, []bool{false}), - }, - - { - info: "table name null", - inputs: []FunctionTestInput{ - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{true}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_json.ToType(), - []string{""}, []bool{false}), - }, - expect: NewFunctionTestResult(types.T_int64.ToType(), true, - []int64{0}, []bool{false}), - }, - - { - info: "vector type null", - inputs: []FunctionTestInput{ - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{0}, []bool{true}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_json.ToType(), - []string{""}, []bool{false}), - }, - expect: NewFunctionTestResult(types.T_int64.ToType(), true, - []int64{0}, []bool{false}), - }, - - { - info: "dimension null", - inputs: []FunctionTestInput{ - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{224}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{true}), - NewFunctionTestInput(types.T_json.ToType(), - []string{""}, []bool{false}), - }, - expect: NewFunctionTestResult(types.T_int64.ToType(), true, - []int64{0}, []bool{false}), - }, - - { - info: "cdc json null", - inputs: []FunctionTestInput{ - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_json.ToType(), - []string{""}, []bool{true}), - }, - expect: NewFunctionTestResult(types.T_int64.ToType(), true, - []int64{0}, []bool{false}), - }, - - { - info: "f32 cdc json invalid", - inputs: []FunctionTestInput{ - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{int32(types.T_array_float32)}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_json.ToType(), - []string{"{..."}, []bool{false}), - }, - expect: NewFunctionTestResult(types.T_int64.ToType(), true, - []int64{0}, []bool{false}), - }, - { - info: "f64 cdc json invalid", - inputs: []FunctionTestInput{ - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{int32(types.T_array_float64)}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_json.ToType(), - []string{"{..."}, []bool{false}), - }, - expect: NewFunctionTestResult(types.T_int64.ToType(), true, - []int64{0}, []bool{false}), - }, - { - info: "invalid vector type", - inputs: []FunctionTestInput{ - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_varchar.ToType(), - []string{""}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{int32(types.T_float32)}, []bool{false}), - NewFunctionTestInput(types.T_int32.ToType(), - []int32{2}, []bool{false}), - NewFunctionTestInput(types.T_json.ToType(), - []string{"{..."}, []bool{false}), - }, - expect: NewFunctionTestResult(types.T_int64.ToType(), true, - []int64{0}, []bool{false}), - }, - } - - proc := testutil.NewProcess(t) - for _, tc := range tcs { - fcTC := NewFunctionTestCase(proc, - tc.inputs, tc.expect, hnswCdcUpdate) - s, info := fcTC.Run() - require.True(t, s, info, tc.info) - } -} diff --git a/pkg/sql/plan/function/function_id.go b/pkg/sql/plan/function/function_id.go index 9f96010c6e306..68decb7e4d65d 100644 --- a/pkg/sql/plan/function/function_id.go +++ b/pkg/sql/plan/function/function_id.go @@ -420,12 +420,9 @@ const ( // hash partition function HASH_PARTITION = 349 - // hnsw function - HNSW_CDC_UPDATE = 350 - // FUNCTION_END_NUMBER is not a function, just a flag to record the max number of function. // TODO: every one should put the new function id in front of this one if you want to make a new function. - FUNCTION_END_NUMBER = 351 + FUNCTION_END_NUMBER = 350 ) // functionIdRegister is what function we have registered already. @@ -768,7 +765,4 @@ var functionIdRegister = map[string]int32{ // fault inject function "fault_inject": FAULT_INJECT, - - // hnsw function - "hnsw_cdc_update": HNSW_CDC_UPDATE, } diff --git a/pkg/sql/plan/function/function_id_test.go b/pkg/sql/plan/function/function_id_test.go index 8c1d946f39ed4..38082db3a6634 100644 --- a/pkg/sql/plan/function/function_id_test.go +++ b/pkg/sql/plan/function/function_id_test.go @@ -402,9 +402,7 @@ var predefinedFunids = map[int]int{ HASH_PARTITION: 349, - HNSW_CDC_UPDATE: 350, - - FUNCTION_END_NUMBER: 351, + FUNCTION_END_NUMBER: 350, } func Test_funids(t *testing.T) { diff --git a/pkg/sql/plan/function/list_builtIn.go b/pkg/sql/plan/function/list_builtIn.go index 0f250e6e73c58..3f810756eb974 100644 --- a/pkg/sql/plan/function/list_builtIn.go +++ b/pkg/sql/plan/function/list_builtIn.go @@ -7054,27 +7054,6 @@ var supportedOthersBuiltIns = []FuncNew{ Overloads: fulltext_expand_overload(types.T_float32), }, - - // function `HNSW_CDC_UPDATE` - { - functionId: HNSW_CDC_UPDATE, - class: plan.Function_STRICT, - layout: STANDARD_FUNCTION, - checkFn: fixedTypeMatch, - - Overloads: []overload{ - { - overloadId: 0, - args: []types.T{types.T_varchar, types.T_varchar, types.T_int32, types.T_int32, types.T_varchar}, - retType: func(parameters []types.Type) types.Type { - return types.T_uint64.ToType() - }, - newOp: func() executeLogicOfOverload { - return hnswCdcUpdate - }, - }, - }, - }, } // fulltext_match supports varchar, char and text. Expand the function signature to all possible combination of input types diff --git a/pkg/txn/client/client.go b/pkg/txn/client/client.go index 1a9811a5100b6..e039b0b0a60d0 100644 --- a/pkg/txn/client/client.go +++ b/pkg/txn/client/client.go @@ -327,8 +327,12 @@ func (client *txnClient) doCreateTxn( op.timestampWaiter = client.timestampWaiter op.AppendEventCallback( ClosedEvent, - client.updateLastCommitTS, - client.closeTxn, + TxnEventCallback{ + Func: client.updateLastCommitTS, + }, + TxnEventCallback{ + Func: client.closeTxn, + }, ) if err := client.openTxn(op); err != nil { @@ -421,7 +425,7 @@ func (client *txnClient) getTxnMode() txn.TxnMode { return txn.TxnMode_Pessimistic } -func (client *txnClient) updateLastCommitTS(event TxnEvent) { +func (client *txnClient) updateLastCommitTS(ctx context.Context, txnOp TxnOperator, event TxnEvent, value any) (err error) { if event.Txn.CommitTS.IsEmpty() { return } @@ -476,7 +480,7 @@ func (client *txnClient) GetLatestCommitTS() timestamp.Timestamp { } func (client *txnClient) SyncLatestCommitTS(ts timestamp.Timestamp) { - client.updateLastCommitTS(TxnEvent{Txn: txn.TxnMeta{CommitTS: ts}}) + client.updateLastCommitTS(context.TODO(), nil, TxnEvent{Txn: txn.TxnMeta{CommitTS: ts}}, nil) if client.timestampWaiter != nil { ctx, cancel := context.WithTimeoutCause(context.Background(), time.Minute*5, moerr.CauseSyncLatestCommitT) defer cancel() @@ -537,7 +541,7 @@ func (client *txnClient) openTxn(op *txnOperator) error { return nil } -func (client *txnClient) closeTxn(event TxnEvent) { +func (client *txnClient) closeTxn(ctx context.Context, txnOp TxnOperator, event TxnEvent, value any) (err error) { txn := event.Txn client.mu.Lock() @@ -583,6 +587,8 @@ func (client *txnClient) closeTxn(event TxnEvent) { zap.String("txn ID", hex.EncodeToString(txn.ID)), zap.String("stack", string(debug.Stack()))) } + + return } func (client *txnClient) addActiveTxnLocked(op *txnOperator) { diff --git a/pkg/txn/client/operator.go b/pkg/txn/client/operator.go index 56ad58e8ba874..9f2a797ee8332 100644 --- a/pkg/txn/client/operator.go +++ b/pkg/txn/client/operator.go @@ -235,7 +235,7 @@ type txnOperator struct { txn txn.TxnMeta cachedWrites map[uint64][]txn.TxnRequest lockTables []lock.LockTable - callbacks map[EventType][]func(TxnEvent) + callbacks map[EventType][]TxnEventCallback retry bool lockSeq uint64 waitLocks map[uint64]Lock @@ -427,6 +427,7 @@ func (tc *txnOperator) waitActive(ctx context.Context) error { }() cost, err := tc.doCostAction( + ctx, time.Time{}, WaitActiveEvent, func() error { @@ -526,6 +527,7 @@ func (tc *txnOperator) UpdateSnapshot( } _, err := tc.doCostAction( + ctx, time.Time{}, UpdateSnapshotEvent, func() error { @@ -599,7 +601,8 @@ func (tc *txnOperator) Read(ctx context.Context, requests []txn.TxnRequest) (*rp } requests = tc.maybeInsertCachedWrites(requests, false) - return tc.trimResponses(tc.handleError(tc.doSend(ctx, requests, false))) + result, err := tc.doSend(ctx, requests, false) + return tc.trimResponses(tc.handleError(ctx, result, err)) } func (tc *txnOperator) Write(ctx context.Context, requests []txn.TxnRequest) (*rpc.SendResult, error) { @@ -614,6 +617,7 @@ func (tc *txnOperator) WriteAndCommit(ctx context.Context, requests []txn.TxnReq } func (tc *txnOperator) Commit(ctx context.Context) (err error) { + if tc.reset.runningSQL.Load() && !tc.markAborted() { tc.logger.Fatal("commit on running txn", zap.String("txnID", hex.EncodeToString(tc.reset.txnID))) @@ -629,11 +633,18 @@ func (tc *txnOperator) Commit(ctx context.Context) (err error) { tc.reset.commitSeq = tc.NextSequence() tc.reset.commitAt = time.Now() - tc.triggerEvent(newEvent(CommitEvent, txnMeta, tc.reset.commitSeq, nil)) + err = tc.triggerEvent(ctx, newEvent(CommitEvent, txnMeta, tc.reset.commitSeq, nil)) + if err != nil { + return + } defer func() { cost := time.Since(tc.reset.commitAt) v2.TxnCNCommitDurationHistogram.Observe(cost.Seconds()) - tc.triggerEvent(newCostEvent(CommitEvent, tc.getTxnMeta(false), tc.reset.commitSeq, err, cost)) + e := tc.triggerEvent(ctx, newCostEvent(CommitEvent, tc.getTxnMeta(false), tc.reset.commitSeq, err, cost)) + if e != nil { + err = errors.Join(e, err) + } + }() } @@ -641,7 +652,7 @@ func (tc *txnOperator) Commit(ctx context.Context) (err error) { tc.mu.Lock() defer tc.mu.Unlock() tc.mu.txn.Status = txn.TxnStatus_Committed - tc.closeLocked() + tc.closeLocked(ctx) return } @@ -654,6 +665,7 @@ func (tc *txnOperator) Commit(ctx context.Context) (err error) { if result != nil { result.Release() } + return } @@ -684,15 +696,15 @@ func (tc *txnOperator) Rollback(ctx context.Context) (err error) { seq := tc.NextSequence() start := time.Now() - tc.triggerEventLocked(newEvent(RollbackEvent, txnMeta, seq, nil)) + tc.triggerEventLocked(ctx, newEvent(RollbackEvent, txnMeta, seq, nil)) defer func() { cost := time.Since(start) - tc.triggerEventLocked(newCostEvent(RollbackEvent, txnMeta, seq, err, cost)) + tc.triggerEventLocked(ctx, newCostEvent(RollbackEvent, txnMeta, seq, err, cost)) }() defer func() { tc.mu.txn.Status = txn.TxnStatus_Aborted - tc.closeLocked() + tc.closeLocked(ctx) }() if tc.needUnlockLocked() { @@ -703,10 +715,12 @@ func (tc *txnOperator) Rollback(ctx context.Context) (err error) { return nil } - result, err := tc.handleError(tc.doSend(ctx, []txn.TxnRequest{{ + sendresult, err := tc.doSend(ctx, []txn.TxnRequest{{ Method: txn.TxnMethod_Rollback, RollbackRequest: &txn.TxnRollbackRequest{}, - }}, true)) + }}, true) + + result, err := tc.handleError(ctx, sendresult, err) if err != nil { if moerr.IsMoErrCode(err, moerr.ErrTxnClosed) { return nil @@ -791,7 +805,8 @@ func (tc *txnOperator) Debug(ctx context.Context, requests []txn.TxnRequest) (*r } requests = tc.maybeInsertCachedWrites(requests, false) - return tc.trimResponses(tc.handleError(tc.doSend(ctx, requests, false))) + result, err := tc.doSend(ctx, requests, false) + return tc.trimResponses(tc.handleError(ctx, result, err)) } func (tc *txnOperator) doWrite(ctx context.Context, requests []txn.TxnRequest, commit bool) (*rpc.SendResult, error) { @@ -813,7 +828,7 @@ func (tc *txnOperator) doWrite(ctx context.Context, requests []txn.TxnRequest, c } tc.mu.Lock() defer func() { - tc.closeLocked() + tc.closeLocked(ctx) tc.mu.Unlock() }() if tc.mu.closed { @@ -868,7 +883,8 @@ func (tc *txnOperator) doWrite(ctx context.Context, requests []txn.TxnRequest, c return nil, tc.reset.commitErr } - resp, err := tc.trimResponses(tc.handleError(tc.doSend(ctx, requests, commit))) + result, err := tc.doSend(ctx, requests, commit) + resp, err := tc.trimResponses(tc.handleError(ctx, result, err)) if err != nil && commit { tc.reset.commitErr = err } @@ -1040,13 +1056,13 @@ func (tc *txnOperator) doSend( return result, nil } -func (tc *txnOperator) handleError(result *rpc.SendResult, err error) (*rpc.SendResult, error) { +func (tc *txnOperator) handleError(ctx context.Context, result *rpc.SendResult, err error) (*rpc.SendResult, error) { if err != nil { return nil, err } for _, resp := range result.Responses { - if err := tc.handleErrorResponse(resp); err != nil { + if err := tc.handleErrorResponse(ctx, resp); err != nil { result.Release() return nil, err } @@ -1054,7 +1070,7 @@ func (tc *txnOperator) handleError(result *rpc.SendResult, err error) (*rpc.Send return result, nil } -func (tc *txnOperator) handleErrorResponse(resp txn.TxnResponse) error { +func (tc *txnOperator) handleErrorResponse(ctx context.Context, resp txn.TxnResponse) error { switch resp.Method { case txn.TxnMethod_Read: if err := tc.checkResponseTxnStatusForReadWrite(resp); err != nil { @@ -1068,6 +1084,7 @@ func (tc *txnOperator) handleErrorResponse(resp txn.TxnResponse) error { return tc.checkTxnError(resp.TxnError, writeTxnErrors) case txn.TxnMethod_Commit: tc.triggerEventLocked( + ctx, newCostEvent( CommitResponseEvent, tc.mu.txn, @@ -1231,6 +1248,7 @@ func (tc *txnOperator) unlock(ctx context.Context) { if tc.mu.txn.IsRCIsolation() && tc.timestampWaiter != nil { cost, err := tc.doCostAction( + ctx, time.Time{}, CommitWaitApplyEvent, func() error { @@ -1248,6 +1266,7 @@ func (tc *txnOperator) unlock(ctx context.Context) { } _, err := tc.doCostAction( + ctx, time.Time{}, UnlockEvent, func() error { @@ -1272,13 +1291,14 @@ func (tc *txnOperator) needUnlockLocked() bool { return tc.lockService != nil } -func (tc *txnOperator) closeLocked() { +func (tc *txnOperator) closeLocked(ctx context.Context) { if !tc.mu.closed { tc.mu.closed = true if tc.reset.commitErr != nil { tc.mu.txn.Status = txn.TxnStatus_Aborted } tc.triggerEventLocked( + ctx, TxnEvent{ Event: ClosedEvent, Txn: tc.mu.txn, @@ -1360,6 +1380,7 @@ func (tc *txnOperator) NextSequence() uint64 { } func (tc *txnOperator) doCostAction( + ctx context.Context, startAt time.Time, event EventType, action func() error, @@ -1375,6 +1396,7 @@ func (tc *txnOperator) doCostAction( } tc.triggerEventLocked( + ctx, newEvent( event, tc.mu.txn, @@ -1384,6 +1406,7 @@ func (tc *txnOperator) doCostAction( err := action() cost := time.Since(startAt) tc.triggerEventLocked( + ctx, newCostEvent( event, tc.mu.txn, diff --git a/pkg/txn/client/operator_events.go b/pkg/txn/client/operator_events.go index 3505a281c28a7..b9f5a21f04301 100644 --- a/pkg/txn/client/operator_events.go +++ b/pkg/txn/client/operator_events.go @@ -15,6 +15,7 @@ package client import ( + "context" "time" "github.com/matrixorigin/matrixone/pkg/pb/txn" @@ -48,31 +49,35 @@ var ( func (tc *txnOperator) AppendEventCallback( event EventType, - callbacks ...func(TxnEvent)) { + callbacks ...TxnEventCallback) { tc.mu.Lock() defer tc.mu.Unlock() if tc.mu.closed { panic("append callback on closed txn") } if tc.mu.callbacks == nil { - tc.mu.callbacks = make(map[EventType][]func(TxnEvent), 1) + tc.mu.callbacks = make(map[EventType][]TxnEventCallback, 1) } tc.mu.callbacks[event] = append(tc.mu.callbacks[event], callbacks...) } -func (tc *txnOperator) triggerEvent(event TxnEvent) { +func (tc *txnOperator) triggerEvent(ctx context.Context, event TxnEvent) error { tc.mu.RLock() defer tc.mu.RUnlock() - tc.triggerEventLocked(event) + return tc.triggerEventLocked(ctx, event) } -func (tc *txnOperator) triggerEventLocked(event TxnEvent) { +func (tc *txnOperator) triggerEventLocked(ctx context.Context, event TxnEvent) (err error) { if tc.mu.callbacks == nil { return } for _, cb := range tc.mu.callbacks[event.Event] { - cb(event) + err = cb.Func(ctx, tc, event, cb.Value) + if err != nil { + return + } } + return } func newCostEvent( diff --git a/pkg/txn/client/operator_events_test.go b/pkg/txn/client/operator_events_test.go index f710e3be928c2..16dea18e6069a 100644 --- a/pkg/txn/client/operator_events_test.go +++ b/pkg/txn/client/operator_events_test.go @@ -46,9 +46,12 @@ func runClosedEventTests( func(ctx context.Context, tc *txnOperator, _ *testTxnSender) { cnt := 0 tc.AppendEventCallback(ClosedEvent, - func(event TxnEvent) { - cnt++ - assert.Equal(t, status, event.Txn.Status) + TxnEventCallback{ + Func: func(ctx context.Context, tc TxnOperator, event TxnEvent, value any) error { + cnt++ + assert.Equal(t, status, event.Txn.Status) + return nil + }, }) require.NoError(t, getAction(tc)(ctx)) assert.Equal(t, 1, cnt) diff --git a/pkg/txn/client/types.go b/pkg/txn/client/types.go index 9904a2b638d02..b4ff5dbd4f5bc 100644 --- a/pkg/txn/client/types.go +++ b/pkg/txn/client/types.go @@ -194,7 +194,7 @@ type TxnOperator interface { // AppendEventCallback append callback. All append callbacks will be called sequentially // if event happen. - AppendEventCallback(event EventType, callbacks ...func(TxnEvent)) + AppendEventCallback(event EventType, callbacks ...TxnEventCallback) // Debug send debug request to DN, after use, SendResult needs to call the Release // method. @@ -326,3 +326,20 @@ func (e TxnEvent) Committed() bool { func (e TxnEvent) Aborted() bool { return e.Txn.Status == txn.TxnStatus_Aborted } + +type TxnEventCallback struct { + Func func(context.Context, TxnOperator, TxnEvent, any) error + Value any +} + +func NewTxnEventCallback(f func(context.Context, TxnOperator, TxnEvent, any) error) TxnEventCallback { + return TxnEventCallback{ + Func: f, + } +} +func NewTxnEventCallbackWithValue(f func(context.Context, TxnOperator, TxnEvent, any) error, v any) TxnEventCallback { + return TxnEventCallback{ + Func: f, + Value: v, + } +} diff --git a/pkg/txn/storage/memorystorage/storage_txn_client.go b/pkg/txn/storage/memorystorage/storage_txn_client.go index 1cb8af945ac49..cf02f63e19196 100644 --- a/pkg/txn/storage/memorystorage/storage_txn_client.go +++ b/pkg/txn/storage/memorystorage/storage_txn_client.go @@ -188,7 +188,7 @@ func (s *StorageTxnOperator) IsOpenLog() bool { panic("unimplemented") } -func (s *StorageTxnOperator) AppendEventCallback(event client.EventType, callbacks ...func(client.TxnEvent)) { +func (s *StorageTxnOperator) AppendEventCallback(event client.EventType, callbacks ...client.TxnEventCallback) { panic("unimplemented") } diff --git a/pkg/txn/trace/service_txn_event.go b/pkg/txn/trace/service_txn_event.go index 8bb6f1d49b8fb..5058efb767600 100644 --- a/pkg/txn/trace/service_txn_event.go +++ b/pkg/txn/trace/service_txn_event.go @@ -70,21 +70,21 @@ func (s *service) TxnCreated(op client.TxnOperator) { } if register { - op.AppendEventCallback(client.WaitActiveEvent, s.handleTxnActive) - op.AppendEventCallback(client.UpdateSnapshotEvent, s.handleTxnUpdateSnapshot) - op.AppendEventCallback(client.CommitEvent, s.handleTxnCommit) - op.AppendEventCallback(client.RollbackEvent, s.handleTxnRollback) - - op.AppendEventCallback(client.CommitResponseEvent, s.handleTxnActionEvent) - op.AppendEventCallback(client.CommitWaitApplyEvent, s.handleTxnActionEvent) - op.AppendEventCallback(client.UnlockEvent, s.handleTxnActionEvent) - op.AppendEventCallback(client.RangesEvent, s.handleTxnActionEvent) - op.AppendEventCallback(client.BuildPlanEvent, s.handleTxnActionEvent) - op.AppendEventCallback(client.ExecuteSQLEvent, s.handleTxnActionEvent) - op.AppendEventCallback(client.CompileEvent, s.handleTxnActionEvent) - op.AppendEventCallback(client.TableScanEvent, s.handleTxnActionEvent) - op.AppendEventCallback(client.WorkspaceWriteEvent, s.handleTxnActionEvent) - op.AppendEventCallback(client.WorkspaceAdjustEvent, s.handleTxnActionEvent) + op.AppendEventCallback(client.WaitActiveEvent, client.NewTxnEventCallback(s.handleTxnActive)) + op.AppendEventCallback(client.UpdateSnapshotEvent, client.NewTxnEventCallback(s.handleTxnUpdateSnapshot)) + op.AppendEventCallback(client.CommitEvent, client.NewTxnEventCallback(s.handleTxnCommit)) + op.AppendEventCallback(client.RollbackEvent, client.NewTxnEventCallback(s.handleTxnRollback)) + + op.AppendEventCallback(client.CommitResponseEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) + op.AppendEventCallback(client.CommitWaitApplyEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) + op.AppendEventCallback(client.UnlockEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) + op.AppendEventCallback(client.RangesEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) + op.AppendEventCallback(client.BuildPlanEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) + op.AppendEventCallback(client.ExecuteSQLEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) + op.AppendEventCallback(client.CompileEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) + op.AppendEventCallback(client.TableScanEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) + op.AppendEventCallback(client.WorkspaceWriteEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) + op.AppendEventCallback(client.WorkspaceAdjustEvent, client.NewTxnEventCallback(s.handleTxnActionEvent)) } } @@ -482,7 +482,7 @@ func (s *service) TxnEventEnabled() bool { return s.atomic.txnEventEnabled.Load() } -func (s *service) handleTxnActive(e client.TxnEvent) { +func (s *service) handleTxnActive(ctx context.Context, txnOp client.TxnOperator, e client.TxnEvent, v any) (err error) { if s.atomic.closed.Load() { return } @@ -496,9 +496,11 @@ func (s *service) handleTxnActive(e client.TxnEvent) { if s.atomic.txnActionEventEnabled.Load() { s.doTxnEventAction(e) } + + return } -func (s *service) handleTxnUpdateSnapshot(e client.TxnEvent) { +func (s *service) handleTxnUpdateSnapshot(ctx context.Context, txnOp client.TxnOperator, e client.TxnEvent, v any) (err error) { if s.atomic.closed.Load() { return } @@ -512,9 +514,10 @@ func (s *service) handleTxnUpdateSnapshot(e client.TxnEvent) { if s.atomic.txnActionEventEnabled.Load() { s.doTxnEventAction(e) } + return } -func (s *service) handleTxnCommit(e client.TxnEvent) { +func (s *service) handleTxnCommit(ctx context.Context, txnOp client.TxnOperator, e client.TxnEvent, v any) (err error) { if s.atomic.closed.Load() { return } @@ -534,9 +537,10 @@ func (s *service) handleTxnCommit(e client.TxnEvent) { if s.atomic.txnActionEventEnabled.Load() { s.doTxnEventAction(e) } + return } -func (s *service) handleTxnRollback(e client.TxnEvent) { +func (s *service) handleTxnRollback(ctx context.Context, txnOp client.TxnOperator, e client.TxnEvent, v any) (err error) { if s.atomic.closed.Load() { return } @@ -556,9 +560,10 @@ func (s *service) handleTxnRollback(e client.TxnEvent) { if s.atomic.txnActionEventEnabled.Load() { s.doTxnEventAction(e) } + return } -func (s *service) handleTxnActionEvent(event client.TxnEvent) { +func (s *service) handleTxnActionEvent(ctx context.Context, txnOp client.TxnOperator, event client.TxnEvent, v any) (err error) { if s.atomic.closed.Load() { return } @@ -566,6 +571,7 @@ func (s *service) handleTxnActionEvent(event client.TxnEvent) { if s.atomic.txnActionEventEnabled.Load() { s.doTxnEventAction(event) } + return } func (s *service) TxnError( diff --git a/pkg/vectorindex/cache/cache.go b/pkg/vectorindex/cache/cache.go index 881540ff6971a..39e78b274b7c5 100644 --- a/pkg/vectorindex/cache/cache.go +++ b/pkg/vectorindex/cache/cache.go @@ -24,7 +24,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/vectorindex" - "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" ) /* @@ -56,8 +56,8 @@ var ( // Various vector index algorithm wants to share with VectorIndexCache need to implement VectorIndexSearchIf interface (see HnswSearch) type VectorIndexSearchIf interface { - Search(proc *process.Process, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) - Load(*process.Process) error + Search(proc *sqlexec.SqlProcess, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) + Load(*sqlexec.SqlProcess) error UpdateConfig(VectorIndexSearchIf) error Destroy() } @@ -84,14 +84,14 @@ func (s *VectorIndexSearch) Destroy() { s.Status.Store(STATUS_DESTROYED) } -func (s *VectorIndexSearch) Load(proc *process.Process) error { +func (s *VectorIndexSearch) Load(sqlproc *sqlexec.SqlProcess) error { s.Mutex.Lock() defer func() { s.Mutex.Unlock() s.Cond.Broadcast() }() - err := s.Algo.Load(proc) + err := s.Algo.Load(sqlproc) if err != nil { // load error s.Status.Store(STATUS_ERROR) @@ -121,7 +121,7 @@ func (s *VectorIndexSearch) extend(update bool) { s.ExpireAt.Store(ts) } -func (s *VectorIndexSearch) Search(proc *process.Process, newalgo VectorIndexSearchIf, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { +func (s *VectorIndexSearch) Search(sqlproc *sqlexec.SqlProcess, newalgo VectorIndexSearchIf, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { s.Cond.L.Lock() defer s.Cond.L.Unlock() @@ -146,7 +146,7 @@ func (s *VectorIndexSearch) Search(proc *process.Process, newalgo VectorIndexSea } s.extend(false) - return s.Algo.Search(proc, query, rt) + return s.Algo.Search(sqlproc, query, rt) } // implementation of VectorIndexCache @@ -248,7 +248,7 @@ func (c *VectorIndexCache) Destroy() { } // Get index from cache and return VectorIndexSearchIf interface -func (c *VectorIndexCache) Search(proc *process.Process, key string, newalgo VectorIndexSearchIf, +func (c *VectorIndexCache) Search(sqlproc *sqlexec.SqlProcess, key string, newalgo VectorIndexSearchIf, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { for { s := &VectorIndexSearch{Algo: newalgo} @@ -258,13 +258,13 @@ func (c *VectorIndexCache) Search(proc *process.Process, key string, newalgo Vec algo := value.(*VectorIndexSearch) if !loaded { // load model from database and if error during loading, remove the entry from gIndexMap - err := algo.Load(proc) + err := algo.Load(sqlproc) if err != nil { c.IndexMap.Delete(key) return nil, nil, err } } - keys, distances, err = algo.Search(proc, newalgo, query, rt) + keys, distances, err = algo.Search(sqlproc, newalgo, query, rt) if err != nil { if moerr.IsMoErrCode(err, moerr.ErrInvalidState) { // index destroyed by Remove() or HouseKeeping. Retry! diff --git a/pkg/vectorindex/cache/cache_test.go b/pkg/vectorindex/cache/cache_test.go index 806032293a239..ce6712b7b7996 100644 --- a/pkg/vectorindex/cache/cache_test.go +++ b/pkg/vectorindex/cache/cache_test.go @@ -24,7 +24,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/testutil" "github.com/matrixorigin/matrixone/pkg/vectorindex" - "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/stretchr/testify/require" usearch "github.com/unum-cloud/usearch/golang" @@ -35,7 +35,7 @@ type MockSearch struct { Tblcfg vectorindex.IndexTableConfig } -func (m *MockSearch) Search(proc *process.Process, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { +func (m *MockSearch) Search(sqlproc *sqlexec.SqlProcess, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { //time.Sleep(2 * time.Millisecond) return []int64{1}, []float64{2.0}, nil } @@ -43,7 +43,7 @@ func (m *MockSearch) Search(proc *process.Process, query any, rt vectorindex.Run func (m *MockSearch) Destroy() { } -func (m *MockSearch) Load(*process.Process) error { +func (m *MockSearch) Load(*sqlexec.SqlProcess) error { //time.Sleep(6 * time.Second) return nil } @@ -57,7 +57,7 @@ type MockAnySearch struct { Tblcfg vectorindex.IndexTableConfig } -func (m *MockAnySearch) Search(proc *process.Process, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { +func (m *MockAnySearch) Search(sqlproc *sqlexec.SqlProcess, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { //time.Sleep(2 * time.Millisecond) return []any{any(1)}, []float64{2.0}, nil } @@ -65,7 +65,7 @@ func (m *MockAnySearch) Search(proc *process.Process, query any, rt vectorindex. func (m *MockAnySearch) Destroy() { } -func (m *MockAnySearch) Load(*process.Process) error { +func (m *MockAnySearch) Load(*sqlexec.SqlProcess) error { //time.Sleep(6 * time.Second) return nil } @@ -80,7 +80,7 @@ type MockSearchLoadError struct { Tblcfg vectorindex.IndexTableConfig } -func (m *MockSearchLoadError) Search(proc *process.Process, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { +func (m *MockSearchLoadError) Search(sqlproc *sqlexec.SqlProcess, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { return []int64{1}, []float64{2.0}, nil } @@ -88,7 +88,7 @@ func (m *MockSearchLoadError) Destroy() { } -func (m *MockSearchLoadError) Load(*process.Process) error { +func (m *MockSearchLoadError) Load(*sqlexec.SqlProcess) error { return moerr.NewInternalErrorNoCtx("Load from database error") } @@ -102,7 +102,7 @@ type MockSearchSearchError struct { Tblcfg vectorindex.IndexTableConfig } -func (m *MockSearchSearchError) Search(proc *process.Process, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { +func (m *MockSearchSearchError) Search(sqlproc *sqlexec.SqlProcess, query any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { return nil, nil, moerr.NewInternalErrorNoCtx("Search error") } @@ -110,7 +110,7 @@ func (m *MockSearchSearchError) Destroy() { } -func (m *MockSearchSearchError) Load(*process.Process) error { +func (m *MockSearchSearchError) Load(*sqlexec.SqlProcess) error { return nil } @@ -120,6 +120,7 @@ func (m *MockSearchSearchError) UpdateConfig(newalgo VectorIndexSearchIf) error func TestCacheServe(t *testing.T) { proc := testutil.NewProcessWithMPool(t, "", mpool.MustNewZero()) + sqlproc := sqlexec.NewSqlProcess(proc) Cache = NewVectorIndexCache() Cache.serve() Cache.serve() @@ -128,7 +129,7 @@ func TestCacheServe(t *testing.T) { tblcfg := vectorindex.IndexTableConfig{DbName: "db", SrcTable: "src", MetadataTable: "__secondary_meta", IndexTable: "__secondary_index"} m := &MockSearch{Idxcfg: idxcfg, Tblcfg: tblcfg} fp32a := []float32{1, 2, 3, 4, 5, 6, 7, 8} - anykeys, distances, err := Cache.Search(proc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) + anykeys, distances, err := Cache.Search(sqlproc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) require.Nil(t, err) if keys, ok := anykeys.([]int64); ok { require.Equal(t, len(keys), 1) @@ -143,6 +144,7 @@ func TestCacheServe(t *testing.T) { func TestCacheAny(t *testing.T) { proc := testutil.NewProcessWithMPool(t, "", mpool.MustNewZero()) + sqlproc := sqlexec.NewSqlProcess(proc) Cache = NewVectorIndexCache() Cache.serve() Cache.serve() @@ -151,7 +153,7 @@ func TestCacheAny(t *testing.T) { tblcfg := vectorindex.IndexTableConfig{DbName: "db", SrcTable: "src", MetadataTable: "__secondary_meta", IndexTable: "__secondary_index"} m := &MockAnySearch{Idxcfg: idxcfg, Tblcfg: tblcfg} fp32a := []float32{1, 2, 3, 4, 5, 6, 7, 8} - anykeys, distances, err := Cache.Search(proc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) + anykeys, distances, err := Cache.Search(sqlproc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) require.Nil(t, err) keys, ok := anykeys.([]any) require.True(t, ok) @@ -166,6 +168,7 @@ func TestCacheAny(t *testing.T) { func TestCache(t *testing.T) { proc := testutil.NewProcessWithMPool(t, "", mpool.MustNewZero()) + sqlproc := sqlexec.NewSqlProcess(proc) VectorIndexCacheTTL = 5 * time.Second VectorIndexCacheTTL = 5 * time.Second @@ -185,7 +188,7 @@ func TestCache(t *testing.T) { m := &MockSearch{Idxcfg: idxcfg, Tblcfg: tblcfg} os.Stderr.WriteString("cache search\n") fp32a := []float32{1, 2, 3, 4, 5, 6, 7, 8} - anykeys1, distances, err := Cache.Search(proc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) + anykeys1, distances, err := Cache.Search(sqlproc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) require.Nil(t, err) if keys1, ok := anykeys1.([]int64); ok { require.Equal(t, len(keys1), 1) @@ -200,7 +203,7 @@ func TestCache(t *testing.T) { // new search m3 := &MockSearch{Idxcfg: idxcfg, Tblcfg: tblcfg} - anykeys2, distances, err := Cache.Search(proc, tblcfg.IndexTable, m3, fp32a, vectorindex.RuntimeConfig{Limit: 4}) + anykeys2, distances, err := Cache.Search(sqlproc, tblcfg.IndexTable, m3, fp32a, vectorindex.RuntimeConfig{Limit: 4}) require.Nil(t, err) if keys2, ok := anykeys2.([]int64); ok { require.Equal(t, len(keys2), 1) @@ -216,6 +219,7 @@ func TestCache(t *testing.T) { func TestCacheConcurrent(t *testing.T) { proc := testutil.NewProcessWithMPool(t, "", mpool.MustNewZero()) + sqlproc := sqlexec.NewSqlProcess(proc) VectorIndexCacheTTL = 2 * time.Second VectorIndexCacheTTL = 2 * time.Second @@ -243,7 +247,7 @@ func TestCacheConcurrent(t *testing.T) { m := &MockSearch{Idxcfg: idxcfg, Tblcfg: tblcfg} //os.Stderr.WriteString("cache search\n") fp32a := []float32{1, 2, 3, 4, 5, 6, 7, 8} - anykeys, distances, err := Cache.Search(proc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) + anykeys, distances, err := Cache.Search(sqlproc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) require.Nil(t, err) if keys, ok := anykeys.([]int64); ok { require.Equal(t, len(keys), 1) @@ -265,6 +269,7 @@ func TestCacheConcurrent(t *testing.T) { func TestCacheConcurrentNewSearchAndDelete(t *testing.T) { proc := testutil.NewProcessWithMPool(t, "", mpool.MustNewZero()) + sqlproc := sqlexec.NewSqlProcess(proc) VectorIndexCacheTTL = 2 * time.Second VectorIndexCacheTTL = 2 * time.Second @@ -292,7 +297,7 @@ func TestCacheConcurrentNewSearchAndDelete(t *testing.T) { m := &MockSearch{Idxcfg: idxcfg, Tblcfg: tblcfg} //os.Stderr.WriteString("cache search\n") fp32a := []float32{1, 2, 3, 4, 5, 6, 7, 8} - anykeys, distances, err := Cache.Search(proc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) + anykeys, distances, err := Cache.Search(sqlproc, tblcfg.IndexTable, m, fp32a, vectorindex.RuntimeConfig{Limit: 4}) require.Nil(t, err) if keys, ok := anykeys.([]int64); ok { require.Equal(t, len(keys), 1) @@ -321,6 +326,7 @@ func TestCacheConcurrentNewSearchAndDelete(t *testing.T) { func TestCacheLoadError(t *testing.T) { proc := testutil.NewProcessWithMPool(t, "", mpool.MustNewZero()) + sqlproc := sqlexec.NewSqlProcess(proc) VectorIndexCacheTTL = 5 * time.Second Cache = NewVectorIndexCache() @@ -338,7 +344,7 @@ func TestCacheLoadError(t *testing.T) { os.Stderr.WriteString("cache getindex\n") m1 := &MockSearchLoadError{Idxcfg: idxcfg, Tblcfg: tblcfg} fp32a := []float32{1, 2, 3, 4, 5, 6, 7, 8} - _, _, err := Cache.Search(proc, tblcfg.IndexTable, m1, fp32a, vectorindex.RuntimeConfig{Limit: 4}) + _, _, err := Cache.Search(sqlproc, tblcfg.IndexTable, m1, fp32a, vectorindex.RuntimeConfig{Limit: 4}) require.NotNil(t, err) os.Stderr.WriteString(fmt.Sprintf("error : %v\n", err)) @@ -350,6 +356,7 @@ func TestCacheLoadError(t *testing.T) { func TestCacheSearchError(t *testing.T) { proc := testutil.NewProcessWithMPool(t, "", mpool.MustNewZero()) + sqlproc := sqlexec.NewSqlProcess(proc) VectorIndexCacheTTL = 5 * time.Second Cache = NewVectorIndexCache() @@ -367,7 +374,7 @@ func TestCacheSearchError(t *testing.T) { os.Stderr.WriteString("cache getindex\n") m1 := &MockSearchSearchError{Idxcfg: idxcfg, Tblcfg: tblcfg} fp32a := []float32{1, 2, 3, 4, 5, 6, 7, 8} - _, _, err := Cache.Search(proc, tblcfg.IndexTable, m1, fp32a, vectorindex.RuntimeConfig{Limit: 4}) + _, _, err := Cache.Search(sqlproc, tblcfg.IndexTable, m1, fp32a, vectorindex.RuntimeConfig{Limit: 4}) require.NotNil(t, err) os.Stderr.WriteString(fmt.Sprintf("error : %v\n", err)) diff --git a/pkg/vectorindex/hnsw/build.go b/pkg/vectorindex/hnsw/build.go index aebb4cfd1d7a3..4ee506bb1c10e 100644 --- a/pkg/vectorindex/hnsw/build.go +++ b/pkg/vectorindex/hnsw/build.go @@ -25,7 +25,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/vectorindex" - "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" ) type HnswBuild[T types.RealNumbers] struct { @@ -48,7 +48,7 @@ type AddItem[T types.RealNumbers] struct { } // create HsnwBuild struct -func NewHnswBuild[T types.RealNumbers](proc *process.Process, uid string, nworker int32, +func NewHnswBuild[T types.RealNumbers](sqlproc *sqlexec.SqlProcess, uid string, nworker int32, cfg vectorindex.IndexConfig, tblcfg vectorindex.IndexTableConfig) (info *HnswBuild[T], err error) { // estimate the number of worker threads @@ -86,7 +86,7 @@ func NewHnswBuild[T types.RealNumbers](proc *process.Process, uid string, nworke var err0 error closed := false for !closed { - closed, err0 = info.addFromChannel(proc) + closed, err0 = info.addFromChannel(sqlproc) if err0 != nil { info.err_chan <- err0 return @@ -99,17 +99,18 @@ func NewHnswBuild[T types.RealNumbers](proc *process.Process, uid string, nworke return info, nil } -func (h *HnswBuild[T]) addFromChannel(proc *process.Process) (stream_closed bool, err error) { +func (h *HnswBuild[T]) addFromChannel(sqlproc *sqlexec.SqlProcess) (stream_closed bool, err error) { var res AddItem[T] var ok bool + procCtx := sqlproc.GetContext() select { case res, ok = <-h.add_chan: if !ok { return true, nil } - case <-proc.Ctx.Done(): - return false, moerr.NewInternalError(proc.Ctx, "context cancelled") + case <-procCtx.Done(): + return false, moerr.NewInternalError(procCtx, "context cancelled") } // add diff --git a/pkg/vectorindex/hnsw/build_test.go b/pkg/vectorindex/hnsw/build_test.go index 102bee2b78f82..ef820eee55775 100644 --- a/pkg/vectorindex/hnsw/build_test.go +++ b/pkg/vectorindex/hnsw/build_test.go @@ -25,6 +25,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/testutil" "github.com/matrixorigin/matrixone/pkg/vectorindex" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/stretchr/testify/require" "golang.org/x/exp/rand" @@ -36,6 +37,7 @@ const MaxIndexCapacity = 100000 func TestBuildMulti(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) ndim := 32 nthread := 8 @@ -55,7 +57,7 @@ func TestBuildMulti(t *testing.T) { IndexCapacity: MaxIndexCapacity} uid := fmt.Sprintf("%s:%d:%d", "localhost", 1, 0) - build, err := NewHnswBuild[float32](proc, uid, 1, idxcfg, tblcfg) + build, err := NewHnswBuild[float32](sqlproc, uid, 1, idxcfg, tblcfg) require.Nil(t, err) defer build.Destroy() @@ -202,6 +204,7 @@ func TestBuildSingleThreadF64(t *testing.T) { func runBuildSingleThread[T types.RealNumbers](t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) ndim := 32 nthread := 2 @@ -228,7 +231,7 @@ func runBuildSingleThread[T types.RealNumbers](t *testing.T) { IndexCapacity: MaxIndexCapacity} uid := fmt.Sprintf("%s:%d:%d", "localhost", 1, 0) - build, err := NewHnswBuild[T](proc, uid, 1, idxcfg, tblcfg) + build, err := NewHnswBuild[T](sqlproc, uid, 1, idxcfg, tblcfg) require.Nil(t, err) defer build.Destroy() diff --git a/pkg/vectorindex/hnsw/model.go b/pkg/vectorindex/hnsw/model.go index eb0eb09b398f3..a3f79e98336cc 100644 --- a/pkg/vectorindex/hnsw/model.go +++ b/pkg/vectorindex/hnsw/model.go @@ -33,7 +33,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/util/executor" "github.com/matrixorigin/matrixone/pkg/vectorindex" - "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" usearch "github.com/unum-cloud/usearch/golang" ) @@ -46,6 +46,7 @@ type HnswModel[T types.RealNumbers] struct { // info required for build MaxCapacity uint + NThread uint // from metadata. info required for search Timestamp int64 @@ -66,29 +67,48 @@ func NewHnswModelForBuild[T types.RealNumbers](id string, cfg vectorindex.IndexC idx := &HnswModel[T]{} idx.Id = id + idx.NThread = uint(nthread) + idx.MaxCapacity = max_capacity - idx.Index, err = usearch.NewIndex(cfg.Usearch) + err = idx.initIndex(cfg) if err != nil { return nil, err } - idx.MaxCapacity = max_capacity + return idx, nil +} + +func (idx *HnswModel[T]) initIndex(cfg vectorindex.IndexConfig) (err error) { + idx.Index, err = usearch.NewIndex(cfg.Usearch) + if err != nil { + return err + } + + defer func() { + if err != nil { + if idx.Index != nil { + idx.Index.Destroy() + idx.Index = nil + } + } + }() err = idx.Index.Reserve(idx.MaxCapacity) if err != nil { - return nil, err + return err } - err = idx.Index.ChangeThreadsAdd(uint(nthread)) + err = idx.Index.ChangeThreadsAdd(idx.NThread) if err != nil { - return nil, err + return err } - err = idx.Index.ChangeThreadsSearch(uint(nthread)) + err = idx.Index.ChangeThreadsSearch(idx.NThread) if err != nil { - return nil, err + return err } - return idx, nil + + return nil } // Destroy the struct @@ -190,6 +210,15 @@ func (idx *HnswModel[T]) SaveToFile() error { idx.Index = nil idx.Path = f.Name() + // Do NOT set filesize here. filesize == 0 means file didn't save to database yet + /* + fi, err := os.Stat(idx.Path) + if err != nil { + return err + } + idx.FileSize := fi.Size() + */ + return nil } @@ -383,13 +412,14 @@ func (idx *HnswModel[T]) Contains(key int64) (found bool, err error) { // load chunk from database func (idx *HnswModel[T]) loadChunkFromBuffer(ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, stream_chan chan executor.Result, error_chan chan error, buffer []byte) (stream_closed bool, err error) { var res executor.Result var ok bool + procCtx := sqlproc.GetContext() select { case res, ok = <-stream_chan: if !ok { @@ -397,8 +427,8 @@ func (idx *HnswModel[T]) loadChunkFromBuffer(ctx context.Context, } case err = <-error_chan: return false, err - case <-proc.Ctx.Done(): - return false, moerr.NewInternalError(proc.Ctx, "context cancelled") + case <-procCtx.Done(): + return false, moerr.NewInternalError(procCtx, "context cancelled") case <-ctx.Done(): return false, moerr.NewInternalErrorf(ctx, "context cancelled: %v", ctx.Err()) } @@ -416,7 +446,7 @@ func (idx *HnswModel[T]) loadChunkFromBuffer(ctx context.Context, } func (idx *HnswModel[T]) LoadIndexFromBuffer( - proc *process.Process, + sqlproc *sqlexec.SqlProcess, idxcfg vectorindex.IndexConfig, tblcfg vectorindex.IndexTableConfig, nthread int64, @@ -433,7 +463,7 @@ func (idx *HnswModel[T]) LoadIndexFromBuffer( } if !view { - return moerr.NewInternalError(proc.Ctx, "LoadIndexFromBuffer only enable when view = true") + return moerr.NewInternalError(sqlproc.GetContext(), "LoadIndexFromBuffer only enable when view = true") } idx.View = true @@ -451,11 +481,11 @@ func (idx *HnswModel[T]) LoadIndexFromBuffer( // run streaming sql sql := fmt.Sprintf("SELECT chunk_id, data from `%s`.`%s` WHERE index_id = '%s'", tblcfg.DbName, tblcfg.IndexTable, idx.Id) - ctx, cancel := context.WithCancelCause(proc.GetTopContext()) + ctx, cancel := context.WithCancelCause(sqlproc.GetTopContext()) defer cancel(nil) go func() { - _, err2 := runSql_streaming(ctx, proc, sql, stream_chan, error_chan) + _, err2 := runSql_streaming(ctx, sqlproc, sql, stream_chan, error_chan) if err2 != nil { error_chan <- err2 return @@ -465,7 +495,7 @@ func (idx *HnswModel[T]) LoadIndexFromBuffer( // incremental load from database sql_closed := false for !sql_closed { - sql_closed, err = idx.loadChunkFromBuffer(ctx, proc, stream_chan, error_chan, idx.buffer) + sql_closed, err = idx.loadChunkFromBuffer(ctx, sqlproc, stream_chan, error_chan, idx.buffer) if err != nil { // notify the producer to stop the sql streaming cancel(err) @@ -497,7 +527,7 @@ func (idx *HnswModel[T]) LoadIndexFromBuffer( chksum := vectorindex.CheckSumFromBuffer(idx.buffer) if chksum != idx.Checksum { - return moerr.NewInternalError(proc.Ctx, "Checksum mismatch with the index file") + return moerr.NewInternalError(sqlproc.GetContext(), "Checksum mismatch with index buffer") } usearchidx, err := usearch.NewIndex(idxcfg.Usearch) @@ -540,13 +570,15 @@ func (idx *HnswModel[T]) LoadIndexFromBuffer( // load chunk from database func (idx *HnswModel[T]) loadChunk(ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, stream_chan chan executor.Result, error_chan chan error, fp *os.File) (stream_closed bool, err error) { var res executor.Result var ok bool + procCtx := sqlproc.GetContext() + select { case res, ok = <-stream_chan: if !ok { @@ -554,8 +586,8 @@ func (idx *HnswModel[T]) loadChunk(ctx context.Context, } case err = <-error_chan: return false, err - case <-proc.Ctx.Done(): - return false, moerr.NewInternalError(proc.Ctx, "context cancelled") + case <-procCtx.Done(): + return false, moerr.NewInternalError(procCtx, "context cancelled") case <-ctx.Done(): return false, moerr.NewInternalErrorf(ctx, "context cancelled: %v", ctx.Err()) } @@ -589,7 +621,7 @@ func (idx *HnswModel[T]) loadChunk(ctx context.Context, // 4. according to the chunk_id, seek to the offset and write the chunk // 5. check the checksum to verify the correctness of the file func (idx *HnswModel[T]) LoadIndex( - proc *process.Process, + sqlproc *sqlexec.SqlProcess, idxcfg vectorindex.IndexConfig, tblcfg vectorindex.IndexTableConfig, nthread int64, @@ -607,7 +639,18 @@ func (idx *HnswModel[T]) LoadIndex( return nil } + if idx.FileSize == 0 && len(idx.Path) == 0 { + // indx is newly created and not save to file yet so simply create a usearch index here + return idx.initIndex(idxcfg) + } + + if len(idx.Checksum) == 0 { + // Checksum is empty. We shouldn't get the file from database + return moerr.NewInternalErrorNoCtx("checksum is empty. Cannot read index file from database") + } + if len(idx.Path) == 0 { + // create tempfile for writing fp, err = os.CreateTemp("", "hnsw") if err != nil { @@ -642,11 +685,11 @@ func (idx *HnswModel[T]) LoadIndex( // run streaming sql sql := fmt.Sprintf("SELECT chunk_id, data from `%s`.`%s` WHERE index_id = '%s'", tblcfg.DbName, tblcfg.IndexTable, idx.Id) - ctx, cancel := context.WithCancelCause(proc.GetTopContext()) + ctx, cancel := context.WithCancelCause(sqlproc.GetTopContext()) defer cancel(nil) go func() { - _, err2 := runSql_streaming(ctx, proc, sql, stream_chan, error_chan) + _, err2 := runSql_streaming(ctx, sqlproc, sql, stream_chan, error_chan) if err2 != nil { error_chan <- err2 return @@ -656,7 +699,7 @@ func (idx *HnswModel[T]) LoadIndex( // incremental load from database sql_closed := false for !sql_closed { - sql_closed, err = idx.loadChunk(ctx, proc, stream_chan, error_chan, fp) + sql_closed, err = idx.loadChunk(ctx, sqlproc, stream_chan, error_chan, fp) if err != nil { // notify the producer to stop the sql streaming cancel(err) @@ -687,6 +730,7 @@ func (idx *HnswModel[T]) LoadIndex( idx.Path = fp.Name() fp.Close() fp = nil + } // check checksum @@ -695,7 +739,7 @@ func (idx *HnswModel[T]) LoadIndex( return err } if chksum != idx.Checksum { - return moerr.NewInternalError(proc.Ctx, "Checksum mismatch with the index file") + return moerr.NewInternalError(sqlproc.GetContext(), "Checksum mismatch with the index file") } usearchidx, err := usearch.NewIndex(idxcfg.Usearch) diff --git a/pkg/vectorindex/hnsw/model_test.go b/pkg/vectorindex/hnsw/model_test.go index 6e0b96260c62d..77b40507afce3 100644 --- a/pkg/vectorindex/hnsw/model_test.go +++ b/pkg/vectorindex/hnsw/model_test.go @@ -26,7 +26,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/testutil" "github.com/matrixorigin/matrixone/pkg/util/executor" "github.com/matrixorigin/matrixone/pkg/vectorindex" - "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/stretchr/testify/require" usearch "github.com/unum-cloud/usearch/golang" @@ -52,7 +52,7 @@ func mock_runSql_streaming(proc *process.Process, sql string, ch chan executor.R // give blob func mock_runSql_streaming_error( ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, sql string, ch chan executor.Result, err_chan chan error, @@ -71,12 +71,13 @@ func TestModelStreamError(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) // stub runSql function runSql = mock_runSql runSql_streaming = mock_runSql_streaming_error - models, err := LoadMetadata[float32](proc, "db", "meta") + models, err := LoadMetadata[float32](sqlproc, "db", "meta") require.Nil(t, err) idxcfg := vectorindex.IndexConfig{Type: "hnsw", Usearch: usearch.DefaultConfig(3)} @@ -88,18 +89,18 @@ func TestModelStreamError(t *testing.T) { defer idx.Destroy() // load from file - err = idx.LoadIndex(proc, idxcfg, tblcfg, 0, false) + err = idx.LoadIndex(sqlproc, idxcfg, tblcfg, 0, false) fmt.Printf("err %v\n", err) require.NotNil(t, err) // load from memory // view == false - err = idx.LoadIndexFromBuffer(proc, idxcfg, tblcfg, 0, false) + err = idx.LoadIndexFromBuffer(sqlproc, idxcfg, tblcfg, 0, false) fmt.Printf("err %v\n", err) require.NotNil(t, err) // error from mock_runSql_streaming_error - err = idx.LoadIndexFromBuffer(proc, idxcfg, tblcfg, 0, true) + err = idx.LoadIndexFromBuffer(sqlproc, idxcfg, tblcfg, 0, true) fmt.Printf("err %v\n", err) require.NotNil(t, err) } @@ -121,12 +122,13 @@ func TestModelFromBuffer(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) // stub runSql function runSql = mock_runSql runSql_streaming = mock_runSql_streaming - models, err := LoadMetadata[float32](proc, "db", "meta") + models, err := LoadMetadata[float32](sqlproc, "db", "meta") require.Nil(t, err) idxcfg := vectorindex.IndexConfig{Type: "hnsw", Usearch: usearch.DefaultConfig(3)} @@ -137,11 +139,11 @@ func TestModelFromBuffer(t *testing.T) { idx := models[0] defer idx.Destroy() - err = idx.LoadIndexFromBuffer(proc, idxcfg, tblcfg, 0, view) + err = idx.LoadIndexFromBuffer(sqlproc, idxcfg, tblcfg, 0, view) require.Nil(t, err) // double LoadIndex - err = idx.LoadIndexFromBuffer(proc, idxcfg, tblcfg, 0, view) + err = idx.LoadIndexFromBuffer(sqlproc, idxcfg, tblcfg, 0, view) require.Nil(t, err) doModelSearchTest[float32](t, idx, 0, fp32a) @@ -166,12 +168,13 @@ func TestModelFromFileViewTrue(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) // stub runSql function runSql = mock_runSql runSql_streaming = mock_runSql_streaming - models, err := LoadMetadata[float32](proc, "db", "meta") + models, err := LoadMetadata[float32](sqlproc, "db", "meta") require.Nil(t, err) idxcfg := vectorindex.IndexConfig{Type: "hnsw", Usearch: usearch.DefaultConfig(3)} @@ -182,11 +185,11 @@ func TestModelFromFileViewTrue(t *testing.T) { idx := models[0] defer idx.Destroy() - err = idx.LoadIndex(proc, idxcfg, tblcfg, 0, view) + err = idx.LoadIndex(sqlproc, idxcfg, tblcfg, 0, view) require.Nil(t, err) // double LoadIndex - err = idx.LoadIndex(proc, idxcfg, tblcfg, 0, view) + err = idx.LoadIndex(sqlproc, idxcfg, tblcfg, 0, view) require.Nil(t, err) doModelSearchTest[float32](t, idx, 0, fp32a) @@ -212,12 +215,13 @@ func TestModel(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) // stub runSql function runSql = mock_runSql runSql_streaming = mock_runSql_streaming - models, err := LoadMetadata[float32](proc, "db", "meta") + models, err := LoadMetadata[float32](sqlproc, "db", "meta") require.Nil(t, err) idxcfg := vectorindex.IndexConfig{Type: "hnsw", Usearch: usearch.DefaultConfig(3)} @@ -228,11 +232,11 @@ func TestModel(t *testing.T) { idx := models[0] defer idx.Destroy() - err = idx.LoadIndex(proc, idxcfg, tblcfg, 0, view) + err = idx.LoadIndex(sqlproc, idxcfg, tblcfg, 0, view) require.Nil(t, err) // double LoadIndex - err = idx.LoadIndex(proc, idxcfg, tblcfg, 0, view) + err = idx.LoadIndex(sqlproc, idxcfg, tblcfg, 0, view) require.Nil(t, err) doModelSearchTest[float32](t, idx, 0, fp32a) @@ -242,7 +246,7 @@ func TestModel(t *testing.T) { err = idx.Unload() require.Nil(t, err) - err = idx.LoadIndex(proc, idxcfg, tblcfg, 0, view) + err = idx.LoadIndex(sqlproc, idxcfg, tblcfg, 0, view) require.Nil(t, err) doModelSearchTest[float32](t, idx, 0, fp32a) @@ -278,7 +282,7 @@ func TestModel(t *testing.T) { err = idx.Unload() require.Nil(t, err) - err = idx.LoadIndex(proc, idxcfg, tblcfg, 0, view) + err = idx.LoadIndex(sqlproc, idxcfg, tblcfg, 0, view) require.Nil(t, err) doModelSearchTest[float32](t, idx, uint64(key), v) @@ -312,7 +316,7 @@ func TestModel(t *testing.T) { require.NotNil(t, err) // load again - err = idx.LoadIndex(proc, idxcfg, tblcfg, 0, view) + err = idx.LoadIndex(sqlproc, idxcfg, tblcfg, 0, view) require.Nil(t, err) key = int64(1000) diff --git a/pkg/vectorindex/hnsw/search.go b/pkg/vectorindex/hnsw/search.go index 560904cad598e..b48d47f4d034a 100644 --- a/pkg/vectorindex/hnsw/search.go +++ b/pkg/vectorindex/hnsw/search.go @@ -27,7 +27,6 @@ import ( "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" "github.com/matrixorigin/matrixone/pkg/vectorindex/metric" "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" - "github.com/matrixorigin/matrixone/pkg/vm/process" ) var runSql = sqlexec.RunSql @@ -69,7 +68,7 @@ func (s *HnswSearch[T]) unlock() { } // Search the hnsw index (implement VectorIndexSearch.Search) -func (s *HnswSearch[T]) Search(proc *process.Process, anyquery any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { +func (s *HnswSearch[T]) Search(sqlproc *sqlexec.SqlProcess, anyquery any, rt vectorindex.RuntimeConfig) (keys any, distances []float64, err error) { query, ok := anyquery.([]T) if !ok { @@ -131,7 +130,7 @@ func (s *HnswSearch[T]) Search(proc *process.Process, anyquery any, rt vectorind srif := heap.Pop() sr, ok := srif.(*vectorindex.SearchResult) if !ok { - return nil, nil, moerr.NewInternalError(proc.Ctx, "heap return key is not int64") + return nil, nil, moerr.NewInternalError(sqlproc.GetContext(), "heap return key is not int64") } reskeys = append(reskeys, sr.Id) sr.Distance = metric.DistanceTransformHnsw(sr.Distance, s.Idxcfg.OpType, s.Idxcfg.Usearch.Metric) @@ -170,10 +169,10 @@ func (s *HnswSearch[T]) Destroy() { } // load metadata from database -func LoadMetadata[T types.RealNumbers](proc *process.Process, dbname string, metatbl string) ([]*HnswModel[T], error) { +func LoadMetadata[T types.RealNumbers](sqlproc *sqlexec.SqlProcess, dbname string, metatbl string) ([]*HnswModel[T], error) { sql := fmt.Sprintf("SELECT * FROM `%s`.`%s` ORDER BY timestamp ASC", dbname, metatbl) - res, err := runSql(proc, sql) + res, err := runSql(sqlproc, sql) if err != nil { return nil, err } @@ -205,11 +204,11 @@ func LoadMetadata[T types.RealNumbers](proc *process.Process, dbname string, met } // load index from database -func (s *HnswSearch[T]) LoadIndex(proc *process.Process, indexes []*HnswModel[T]) ([]*HnswModel[T], error) { +func (s *HnswSearch[T]) LoadIndex(sqlproc *sqlexec.SqlProcess, indexes []*HnswModel[T]) ([]*HnswModel[T], error) { var err error for _, idx := range indexes { - err = idx.LoadIndexFromBuffer(proc, s.Idxcfg, s.Tblcfg, s.ThreadsSearch, true) + err = idx.LoadIndexFromBuffer(sqlproc, s.Idxcfg, s.Tblcfg, s.ThreadsSearch, true) if err != nil { break } @@ -226,16 +225,16 @@ func (s *HnswSearch[T]) LoadIndex(proc *process.Process, indexes []*HnswModel[T] } // load index from database (implement VectorIndexSearch.LoadFromDatabase) -func (s *HnswSearch[T]) Load(proc *process.Process) error { +func (s *HnswSearch[T]) Load(sqlproc *sqlexec.SqlProcess) error { // load metadata - indexes, err := LoadMetadata[T](proc, s.Tblcfg.DbName, s.Tblcfg.MetadataTable) + indexes, err := LoadMetadata[T](sqlproc, s.Tblcfg.DbName, s.Tblcfg.MetadataTable) if err != nil { return err } if len(indexes) > 0 { // load index model - indexes, err = s.LoadIndex(proc, indexes) + indexes, err = s.LoadIndex(sqlproc, indexes) if err != nil { return err } diff --git a/pkg/vectorindex/hnsw/search_test.go b/pkg/vectorindex/hnsw/search_test.go index a608e68509ae0..5274cd307ca54 100644 --- a/pkg/vectorindex/hnsw/search_test.go +++ b/pkg/vectorindex/hnsw/search_test.go @@ -34,6 +34,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/util/executor" "github.com/matrixorigin/matrixone/pkg/vectorindex" "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/matrixorigin/matrixone/pkg/vm/process" "github.com/stretchr/testify/require" @@ -41,20 +42,21 @@ import ( ) // give metadata [index_id, checksum, timestamp] -func mock_runSql(proc *process.Process, sql string) (executor.Result, error) { - +func mock_runSql(sqlproc *sqlexec.SqlProcess, sql string) (executor.Result, error) { + proc := sqlproc.Proc return executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{makeMetaBatch(proc)}}, nil } // give blob func mock_runSql_streaming( ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, sql string, ch chan executor.Result, err_chan chan error, ) (executor.Result, error) { + proc := sqlproc.Proc defer close(ch) res := executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{makeIndexBatch(proc)}} ch <- res @@ -62,15 +64,15 @@ func mock_runSql_streaming( } // give metadata [index_id, checksum, timestamp] -func mock_runSql_2files(proc *process.Process, sql string) (executor.Result, error) { - +func mock_runSql_2files(sqlproc *sqlexec.SqlProcess, sql string) (executor.Result, error) { + proc := sqlproc.Proc return executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{makeMetaBatch2Files(proc)}}, nil } // give blob func mock_runSql_streaming_2files( ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, sql string, ch chan executor.Result, err_chan chan error, @@ -82,6 +84,7 @@ func mock_runSql_streaming_2files( idx = 1 } + proc := sqlproc.Proc defer close(ch) res := executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{makeIndexBatch2Files(proc, idx)}} ch <- res @@ -89,20 +92,22 @@ func mock_runSql_streaming_2files( } // give moindexes metadata [index_table_name, algo_table_type, algo_params, column_name] -func mock_runCatalogSql(proc *process.Process, sql string) (executor.Result, error) { - +func mock_runCatalogSql(sqlproc *sqlexec.SqlProcess, sql string) (executor.Result, error) { + proc := sqlproc.Proc return executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{makeMoIndexesBatch(proc)}}, nil } // give moindexes metadata [index_table_name, algo_table_type, algo_params, column_name] -func mock_runEmptyCatalogSql(proc *process.Process, sql string) (executor.Result, error) { +func mock_runEmptyCatalogSql(sqlproc *sqlexec.SqlProcess, sql string) (executor.Result, error) { + proc := sqlproc.Proc return executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{}}, nil } func TestHnsw(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) // stub runSql function runSql = mock_runSql @@ -131,7 +136,7 @@ func TestHnsw(t *testing.T) { cache.Cache.Once() algo := NewHnswSearch[float32](idxcfg, tblcfg) - anykeys, distances, err := cache.Cache.Search(proc, tblcfg.IndexTable, algo, fp32a, vectorindex.RuntimeConfig{Limit: 4}) + anykeys, distances, err := cache.Cache.Search(sqlproc, tblcfg.IndexTable, algo, fp32a, vectorindex.RuntimeConfig{Limit: 4}) require.Nil(t, err) keys, ok := anykeys.([]int64) require.True(t, ok) diff --git a/pkg/vectorindex/hnsw/sync.go b/pkg/vectorindex/hnsw/sync.go index b18eeec7afc55..5fc5bb3b5f34c 100644 --- a/pkg/vectorindex/hnsw/sync.go +++ b/pkg/vectorindex/hnsw/sync.go @@ -34,7 +34,6 @@ import ( veccache "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" "github.com/matrixorigin/matrixone/pkg/vectorindex/metric" "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" - "github.com/matrixorigin/matrixone/pkg/vm/process" ) // CdcSync is the main function to update hnsw index via CDC. SQL function hnsw_cdc_update() will call this function. @@ -46,25 +45,64 @@ const ( var runTxn = sqlexec.RunTxn var runCatalogSql = sqlexec.RunSql -func CdcSync[T types.RealNumbers](proc *process.Process, db string, tbl string, vectype int32, dimension int32, cdc *vectorindex.VectorIndexCdc[T]) error { +type HnswSync[T types.RealNumbers] struct { + indexes []*HnswModel[T] + idxcfg vectorindex.IndexConfig + tblcfg vectorindex.IndexTableConfig + uid string + ts int64 + ninsert atomic.Int32 + ndelete atomic.Int32 + nupdate atomic.Int32 + current *HnswModel[T] + last *HnswModel[T] +} + +func (s *HnswSync[T]) RunOnce(sqlproc *sqlexec.SqlProcess, cdc *vectorindex.VectorIndexCdc[T]) (err error) { + + defer s.Destroy() + err = s.Update(sqlproc, cdc) + if err != nil { + return err + } - accountId, err := defines.GetAccountId(proc.Ctx) + err = s.Save(sqlproc) if err != nil { return err } + return nil +} + +func NewHnswSync[T types.RealNumbers](sqlproc *sqlexec.SqlProcess, + db string, + tbl string, + vectype int32, + dimension int32) (*HnswSync[T], error) { + var err error + + accountId := uint32(0) + if sqlproc.Proc != nil { + accountId, err = defines.GetAccountId(sqlproc.GetContext()) + if err != nil { + return nil, err + } + } else { + accountId = sqlproc.SqlCtx.AccountId + } + // get index catalog sql := fmt.Sprintf(catalogsql, tbl, db, accountId) - res, err := runCatalogSql(proc, sql) + res, err := runCatalogSql(sqlproc, sql) if err != nil { - return err + return nil, err } defer res.Close() //os.Stderr.WriteString(sql) if len(res.Batches) == 0 { - return moerr.NewInternalError(proc.Ctx, fmt.Sprintf("hnsw cdc sync: no secondary index tables found with accountID %d, table %s and db %s", + return nil, moerr.NewInternalError(sqlproc.GetContext(), fmt.Sprintf("hnsw cdc sync: no secondary index tables found with accountID %d, table %s and db %s", accountId, tbl, db)) } @@ -82,16 +120,16 @@ func CdcSync[T types.RealNumbers](proc *process.Process, db string, tbl string, idxtblcfg.SrcTable = tbl // GetResolveVariableFunc() is nil because of internal SQL proc don't have ResolveVariableFunc(). - if proc.GetResolveVariableFunc() != nil { - val, err := proc.GetResolveVariableFunc()("hnsw_threads_build", true, false) + if sqlproc.GetResolveVariableFunc() != nil { + val, err := sqlproc.GetResolveVariableFunc()("hnsw_threads_build", true, false) if err != nil { - return err + return nil, err } idxtblcfg.ThreadsBuild = vectorindex.GetConcurrencyForBuild(val.(int64)) - idxcap, err := proc.GetResolveVariableFunc()("hnsw_max_index_capacity", true, false) + idxcap, err := sqlproc.GetResolveVariableFunc()("hnsw_max_index_capacity", true, false) if err != nil { - return err + return nil, err } idxtblcfg.IndexCapacity = idxcap.(int64) } else { @@ -113,7 +151,7 @@ func CdcSync[T types.RealNumbers](proc *process.Process, db string, tbl string, if len(paramstr) > 0 { err := json.Unmarshal([]byte(paramstr), ¶m) if err != nil { - return err + return nil, err } } } @@ -134,13 +172,13 @@ func CdcSync[T types.RealNumbers](proc *process.Process, db string, tbl string, idxcfg.Usearch.Quantization, err = QuantizationToUsearch(vectype) if err != nil { - return err + return nil, err } if len(param.M) > 0 { val, err := strconv.Atoi(param.M) if err != nil { - return err + return nil, err } idxcfg.Usearch.Connectivity = uint(val) } @@ -148,14 +186,14 @@ func CdcSync[T types.RealNumbers](proc *process.Process, db string, tbl string, // default L2Sq metrictype, ok := metric.OpTypeToUsearchMetric[param.OpType] if !ok { - return moerr.NewInternalError(proc.Ctx, "Invalid op_type") + return nil, moerr.NewInternalError(sqlproc.GetContext(), "Invalid op_type") } idxcfg.Usearch.Metric = metrictype if len(param.EfConstruction) > 0 { val, err := strconv.Atoi(param.EfConstruction) if err != nil { - return err + return nil, err } idxcfg.Usearch.ExpansionAdd = uint(val) } @@ -163,7 +201,7 @@ func CdcSync[T types.RealNumbers](proc *process.Process, db string, tbl string, if len(param.EfSearch) > 0 { val, err := strconv.Atoi(param.EfSearch) if err != nil { - return err + return nil, err } idxcfg.Usearch.ExpansionSearch = uint(val) } @@ -172,56 +210,56 @@ func CdcSync[T types.RealNumbers](proc *process.Process, db string, tbl string, //os.Stderr.WriteString(fmt.Sprintf("idxcfg: %v\n", idxcfg)) // load metadata - indexes, err := LoadMetadata[T](proc, idxtblcfg.DbName, idxtblcfg.MetadataTable) + indexes, err := LoadMetadata[T](sqlproc, idxtblcfg.DbName, idxtblcfg.MetadataTable) if err != nil { - return err + return nil, err } // assume CDC run in single thread // model id for CDC is cdc:1:0:timestamp uid := fmt.Sprintf("%s:%d:%d", "cdc", 1, 0) ts := time.Now().Unix() - sync := &HnswSync[T]{indexes: indexes, idxcfg: idxcfg, tblcfg: idxtblcfg, cdc: cdc, uid: uid, ts: ts} - defer sync.destroy() - err = sync.run(proc) + sync := &HnswSync[T]{indexes: indexes, idxcfg: idxcfg, tblcfg: idxtblcfg, uid: uid, ts: ts} + + // save all model to local by LoadIndex and Unload + err = sync.DownloadAll(sqlproc) if err != nil { - return err + return nil, err } - // clear the cache (it only work in standalone mode though) - veccache.Cache.Remove(idxtblcfg.IndexTable) - - return nil -} - -type HnswSync[T types.RealNumbers] struct { - indexes []*HnswModel[T] - idxcfg vectorindex.IndexConfig - tblcfg vectorindex.IndexTableConfig - cdc *vectorindex.VectorIndexCdc[T] - uid string - ts int64 - ninsert atomic.Int32 - ndelete atomic.Int32 - nupdate atomic.Int32 - current *HnswModel[T] - last *HnswModel[T] + return sync, nil } -func (s *HnswSync[T]) destroy() { +func (s *HnswSync[T]) Destroy() { for _, m := range s.indexes { m.Destroy() } s.indexes = nil } -func (s *HnswSync[T]) checkContains(proc *process.Process) (maxcap uint, midx []int, err error) { +func (s *HnswSync[T]) DownloadAll(sqlproc *sqlexec.SqlProcess) (err error) { + + for _, m := range s.indexes { + err = m.LoadIndex(sqlproc, s.idxcfg, s.tblcfg, s.tblcfg.ThreadsBuild, false) + if err != nil { + return + } + err = m.Unload() + if err != nil { + return + } + } + + return +} + +func (s *HnswSync[T]) checkContains(sqlproc *sqlexec.SqlProcess, cdc *vectorindex.VectorIndexCdc[T]) (maxcap uint, midx []int, err error) { err_chan := make(chan error, s.tblcfg.ThreadsBuild) maxcap = uint(s.tblcfg.IndexCapacity) // try to find index cap - cdclen := len(s.cdc.Data) + cdclen := len(cdc.Data) midx = make([]int, cdclen) // reset idx to -1 @@ -231,7 +269,7 @@ func (s *HnswSync[T]) checkContains(proc *process.Process) (maxcap uint, midx [] // find corresponding indexes for i, m := range s.indexes { - err = m.LoadIndex(proc, s.idxcfg, s.tblcfg, s.tblcfg.ThreadsBuild, false) + err = m.LoadIndex(sqlproc, s.idxcfg, s.tblcfg, s.tblcfg.ThreadsBuild, false) if err != nil { return 0, nil, err } @@ -247,7 +285,7 @@ func (s *HnswSync[T]) checkContains(proc *process.Process) (maxcap uint, midx [] wg.Add(1) go func(tid int) { defer wg.Done() - for j, row := range s.cdc.Data { + for j, row := range cdc.Data { if j%nthread != tid { continue @@ -288,7 +326,7 @@ func (s *HnswSync[T]) checkContains(proc *process.Process) (maxcap uint, midx [] return maxcap, midx, nil } -func (s *HnswSync[T]) insertAllInParallel(proc *process.Process, maxcap uint, midx []int) error { +func (s *HnswSync[T]) insertAllInParallel(sqlproc *sqlexec.SqlProcess, maxcap uint, midx []int, cdc *vectorindex.VectorIndexCdc[T]) error { var mu sync.Mutex var wg sync.WaitGroup err_chan := make(chan error, s.tblcfg.ThreadsBuild) @@ -299,7 +337,7 @@ func (s *HnswSync[T]) insertAllInParallel(proc *process.Process, maxcap uint, mi go func(tid int) { defer wg.Done() - for j, row := range s.cdc.Data { + for j, row := range cdc.Data { if j%nthread != tid { continue @@ -319,7 +357,7 @@ func (s *HnswSync[T]) insertAllInParallel(proc *process.Process, maxcap uint, mi // make sure last model won't unload when full and return full // don't unload any model here. Quite dangerous and There is no harm not to unload because // cdc size max is 8192. Model will eventually unload when save. - last, _, err := s.getLastModelAndIncrForSync(proc, maxcap, &mu) + last, _, err := s.getLastModelAndIncrForSync(sqlproc, maxcap, &mu) if err != nil { err_chan <- err return @@ -340,7 +378,7 @@ func (s *HnswSync[T]) insertAllInParallel(proc *process.Process, maxcap uint, mi return nil } -func (s *HnswSync[T]) setupModel(proc *process.Process, maxcap uint) error { +func (s *HnswSync[T]) setupModel(sqlproc *sqlexec.SqlProcess, maxcap uint) error { s.current = (*HnswModel[T])(nil) s.last = (*HnswModel[T])(nil) @@ -371,7 +409,7 @@ func (s *HnswSync[T]) setupModel(proc *process.Process, maxcap uint) error { } else { //os.Stderr.WriteString(fmt.Sprintf("load model with index %d\n", len(s.indexes)-1)) // load last - s.last.LoadIndex(proc, s.idxcfg, s.tblcfg, s.tblcfg.ThreadsBuild, false) + s.last.LoadIndex(sqlproc, s.idxcfg, s.tblcfg, s.tblcfg.ThreadsBuild, false) } } @@ -379,15 +417,15 @@ func (s *HnswSync[T]) setupModel(proc *process.Process, maxcap uint) error { return nil } -func (s *HnswSync[T]) sequentialUpdate(proc *process.Process, maxcap uint, midx []int) error { +func (s *HnswSync[T]) sequentialUpdate(sqlproc *sqlexec.SqlProcess, maxcap uint, midx []int, cdc *vectorindex.VectorIndexCdc[T]) error { - for i, row := range s.cdc.Data { + for i, row := range cdc.Data { switch row.Type { case vectorindex.CDC_UPSERT: if midx[i] == -1 { // cannot find key from existing model. simple insert - last, err := s.getLastModel(proc, maxcap) + last, err := s.getLastModel(sqlproc, maxcap) if err != nil { return err } @@ -400,7 +438,7 @@ func (s *HnswSync[T]) sequentialUpdate(proc *process.Process, maxcap uint, midx break } - current, err := s.getCurrentModel(proc, midx[i]) + current, err := s.getCurrentModel(sqlproc, midx[i]) if err != nil { return err } @@ -423,7 +461,7 @@ func (s *HnswSync[T]) sequentialUpdate(proc *process.Process, maxcap uint, midx continue } - current, err := s.getCurrentModel(proc, midx[i]) + current, err := s.getCurrentModel(sqlproc, midx[i]) if err != nil { return err } @@ -435,7 +473,7 @@ func (s *HnswSync[T]) sequentialUpdate(proc *process.Process, maxcap uint, midx } case vectorindex.CDC_INSERT: - last, err := s.getLastModel(proc, maxcap) + last, err := s.getLastModel(sqlproc, maxcap) if err != nil { return err } @@ -451,13 +489,13 @@ func (s *HnswSync[T]) sequentialUpdate(proc *process.Process, maxcap uint, midx return nil } -func (s *HnswSync[T]) run(proc *process.Process) error { +func (s *HnswSync[T]) Update(sqlproc *sqlexec.SqlProcess, cdc *vectorindex.VectorIndexCdc[T]) error { var err error start := time.Now() // check contains and find the correspoding index id - maxcap, midx, err := s.checkContains(proc) + maxcap, midx, err := s.checkContains(sqlproc, cdc) if err != nil { return err } @@ -466,28 +504,28 @@ func (s *HnswSync[T]) run(proc *process.Process) error { checkidxElapsed := t.Sub(start) - s.ninsert.Store(int32(len(s.cdc.Data)) - s.nupdate.Load() - s.ndelete.Load()) + s.ninsert.Store(int32(len(cdc.Data)) - s.nupdate.Load() - s.ndelete.Load()) // setup s.last and s.current model. s.late will point to the last model in metadata and s.current is nil - err = s.setupModel(proc, maxcap) + err = s.setupModel(sqlproc, maxcap) if err != nil { return err } logutil.Infof("hnsw_cdc_update: db=%s, table=%s, cdc: len=%d, ninsert = %d, ndelete = %d, nupdate = %d\n", s.tblcfg.DbName, s.tblcfg.SrcTable, - len(s.cdc.Data), s.ninsert.Load(), s.ndelete.Load(), s.nupdate.Load()) + len(cdc.Data), s.ninsert.Load(), s.ndelete.Load(), s.nupdate.Load()) - if len(s.cdc.Data) == int(s.ninsert.Load()) { + if len(cdc.Data) == int(s.ninsert.Load()) { // pure insert and insert into parallel - err = s.insertAllInParallel(proc, maxcap, midx) + err = s.insertAllInParallel(sqlproc, maxcap, midx, cdc) if err != nil { return err } } else { // perform sequential update in single thread - err = s.sequentialUpdate(proc, maxcap, midx) + err = s.sequentialUpdate(sqlproc, maxcap, midx, cdc) if err != nil { return err } @@ -496,6 +534,14 @@ func (s *HnswSync[T]) run(proc *process.Process) error { t2 := time.Now() updateElapsed := t2.Sub(t) + t3 := time.Now() + saveElapsed := t3.Sub(t2) + logutil.Debugf("hnsw_cdc_update: time elapsed: checkidx %d ms, update %d ms, save %d ms", + checkidxElapsed.Milliseconds(), updateElapsed.Milliseconds(), saveElapsed.Milliseconds()) + return nil +} + +func (s *HnswSync[T]) Save(sqlproc *sqlexec.SqlProcess) error { // save to files and then save to database sqls, err := s.ToSql(s.ts) if err != nil { @@ -506,27 +552,25 @@ func (s *HnswSync[T]) run(proc *process.Process) error { return nil } - err = s.runSqls(proc, sqls) + err = s.runSqls(sqlproc, sqls) if err != nil { return err } - t3 := time.Now() - saveElapsed := t3.Sub(t2) + // clear the cache (it only work in standalone mode though) + veccache.Cache.Remove(s.tblcfg.IndexTable) - logutil.Debugf("hnsw_cdc_update: time elapsed: checkidx %d ms, update %d ms, save %d ms", - checkidxElapsed.Milliseconds(), updateElapsed.Milliseconds(), saveElapsed.Milliseconds()) return nil } -func (s *HnswSync[T]) runSqls(proc *process.Process, sqls []string) error { +func (s *HnswSync[T]) runSqls(sqlproc *sqlexec.SqlProcess, sqls []string) error { /* for _, s := range sqls { os.Stderr.WriteString(fmt.Sprintf("sql : %s\n", s)) } */ opts := executor.Options{} - err := runTxn(proc, func(exec executor.TxnExecutor) error { + err := runTxn(sqlproc, func(exec executor.TxnExecutor) error { for _, sql := range sqls { res, err := exec.Exec(sql, opts.StatementOption()) if err != nil { @@ -549,20 +593,20 @@ func (s *HnswSync[T]) getModelId() string { return id } -func (s *HnswSync[T]) getCurrentModel(proc *process.Process, idx int) (*HnswModel[T], error) { +func (s *HnswSync[T]) getCurrentModel(sqlproc *sqlexec.SqlProcess, idx int) (*HnswModel[T], error) { m := s.indexes[idx] if s.current != m { // check current == last, if not, safe to unload if s.current != nil && s.current != s.last { s.current.Unload() } - m.LoadIndex(proc, s.idxcfg, s.tblcfg, s.tblcfg.ThreadsBuild, false) + m.LoadIndex(sqlproc, s.idxcfg, s.tblcfg, s.tblcfg.ThreadsBuild, false) s.current = m } return s.current, nil } -func (s *HnswSync[T]) getLastModel(proc *process.Process, maxcap uint) (*HnswModel[T], error) { +func (s *HnswSync[T]) getLastModel(sqlproc *sqlexec.SqlProcess, maxcap uint) (*HnswModel[T], error) { full, err := s.last.Full() if err != nil { @@ -589,7 +633,7 @@ func (s *HnswSync[T]) getLastModel(proc *process.Process, maxcap uint) (*HnswMod return s.last, nil } -func (s *HnswSync[T]) getLastModelAndIncrForSync(proc *process.Process, maxcap uint, mu *sync.Mutex) (*HnswModel[T], bool, error) { +func (s *HnswSync[T]) getLastModelAndIncrForSync(sqlproc *sqlexec.SqlProcess, maxcap uint, mu *sync.Mutex) (*HnswModel[T], bool, error) { mu.Lock() defer mu.Unlock() diff --git a/pkg/vectorindex/hnsw/sync_test.go b/pkg/vectorindex/hnsw/sync_test.go index 019f7b5c2b24c..cc3f381d274bf 100644 --- a/pkg/vectorindex/hnsw/sync_test.go +++ b/pkg/vectorindex/hnsw/sync_test.go @@ -15,7 +15,6 @@ package hnsw import ( - "fmt" "testing" "time" @@ -27,7 +26,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/txn/client" "github.com/matrixorigin/matrixone/pkg/util/executor" "github.com/matrixorigin/matrixone/pkg/vectorindex" - "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/stretchr/testify/require" "golang.org/x/exp/rand" ) @@ -70,12 +69,12 @@ func (e *MockTxnExecutor) Txn() client.TxnOperator { } // give metadata [index_id, checksum, timestamp] -func mock_runSql_empty(proc *process.Process, sql string) (executor.Result, error) { - +func mock_runSql_empty(sqlproc *sqlexec.SqlProcess, sql string) (executor.Result, error) { + proc := sqlproc.Proc return executor.Result{Mp: proc.Mp(), Batches: []*batch.Batch{}}, nil } -func mock_runTxn(proc *process.Process, fn func(exec executor.TxnExecutor) error) error { +func mock_runTxn(sqlproc *sqlexec.SqlProcess, fn func(exec executor.TxnExecutor) error) error { exec := &MockTxnExecutor{} err := fn(exec) return err @@ -84,17 +83,19 @@ func mock_runTxn(proc *process.Process, fn func(exec executor.TxnExecutor) error func TestSyncRunSqls(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) + runTxn = mock_runTxn sync := &HnswSync[float32]{} - defer sync.destroy() + defer sync.Destroy() sqls := []string{"fake"} - err := sync.runSqls(proc, sqls) + err := sync.runSqls(sqlproc, sqls) require.NotNil(t, err) sqls = []string{"sql"} - err = sync.runSqls(proc, sqls) + err = sync.runSqls(sqlproc, sqls) require.Nil(t, err) } @@ -102,6 +103,7 @@ func TestSyncEmptyCatalogError(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql_empty runSql_streaming = mock_runSql_streaming @@ -122,7 +124,7 @@ func TestSyncEmptyCatalogError(t *testing.T) { } } - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + _, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) require.NotNil(t, err) } @@ -130,6 +132,7 @@ func TestSyncUpsertWithEmpty(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql_empty runSql_streaming = mock_runSql_streaming @@ -150,13 +153,16 @@ func TestSyncUpsertWithEmpty(t *testing.T) { } } - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } func TestSyncVariableError(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql_2files runSql_streaming = mock_runSql_streaming_2files @@ -186,10 +192,12 @@ func TestSyncVariableError(t *testing.T) { } }) - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) - fmt.Println(err) + _, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) require.NotNil(t, err) + //err = sync.RunOnce(sqlproc, &cdc) + // require.Nil(t, err) + proc.SetResolveVariableFunc(func(key string, b1 bool, b2 bool) (any, error) { switch key { case "hnsw_max_index_capacity": @@ -202,8 +210,7 @@ func TestSyncVariableError(t *testing.T) { } }) - err = CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) - //fmt.Println(err) + _, err = NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) require.NotNil(t, err) } @@ -211,6 +218,7 @@ func TestSyncUpsert(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql runSql_streaming = mock_runSql_streaming @@ -231,7 +239,9 @@ func TestSyncUpsert(t *testing.T) { } } - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } @@ -240,6 +250,7 @@ func TestSyncDelete(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql runSql_streaming = mock_runSql_streaming @@ -256,7 +267,9 @@ func TestSyncDelete(t *testing.T) { key += 1 } - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } @@ -265,6 +278,7 @@ func TestSyncDeleteAndInsert(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql runSql_streaming = mock_runSql_streaming @@ -290,7 +304,9 @@ func TestSyncDeleteAndInsert(t *testing.T) { } - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } @@ -299,6 +315,7 @@ func TestSyncUpdate(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql runSql_streaming = mock_runSql_streaming @@ -316,7 +333,9 @@ func TestSyncUpdate(t *testing.T) { key += 1 } - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } @@ -325,6 +344,7 @@ func TestSyncDeleteAndUpsert(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql runSql_streaming = mock_runSql_streaming @@ -350,7 +370,9 @@ func TestSyncDeleteAndUpsert(t *testing.T) { } - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } @@ -359,6 +381,7 @@ func TestSyncAddOneModel(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql runSql_streaming = mock_runSql_streaming @@ -379,7 +402,9 @@ func TestSyncAddOneModel(t *testing.T) { } } - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } @@ -388,6 +413,7 @@ func TestSyncDelete2Files(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql_2files runSql_streaming = mock_runSql_streaming_2files @@ -404,7 +430,9 @@ func TestSyncDelete2Files(t *testing.T) { key += 1 } - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } @@ -413,6 +441,7 @@ func TestSyncDeleteShuffle2Files(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql_2files runSql_streaming = mock_runSql_streaming_2files @@ -432,7 +461,9 @@ func TestSyncDeleteShuffle2Files(t *testing.T) { rand.Seed(uint64(time.Now().UnixNano())) rand.Shuffle(len(cdc.Data), func(i, j int) { cdc.Data[i], cdc.Data[j] = cdc.Data[j], cdc.Data[i] }) - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } @@ -441,6 +472,7 @@ func TestSyncUpdateShuffle2Files(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql_2files runSql_streaming = mock_runSql_streaming_2files @@ -461,7 +493,9 @@ func TestSyncUpdateShuffle2Files(t *testing.T) { rand.Seed(uint64(time.Now().UnixNano())) rand.Shuffle(len(cdc.Data), func(i, j int) { cdc.Data[i], cdc.Data[j] = cdc.Data[j], cdc.Data[i] }) - err := CdcSync[float32](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[float32](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } @@ -470,6 +504,7 @@ func TestSyncUpdateShuffle2Files(t *testing.T) { func runSyncUpdateInsertShuffle2Files[T types.RealNumbers](t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) runSql = mock_runSql_2files runSql_streaming = mock_runSql_streaming_2files @@ -494,10 +529,14 @@ func runSyncUpdateInsertShuffle2Files[T types.RealNumbers](t *testing.T) { var ff T switch any(ff).(type) { case float32: - err := CdcSync[T](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[T](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) case float64: - err := CdcSync[T](proc, "db", "src", int32(types.T_array_float64), 3, &cdc) + sync, err := NewHnswSync[T](sqlproc, "db", "src", int32(types.T_array_float64), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) } } @@ -513,6 +552,7 @@ func TestSyncUpdateInsertShuffle2FilesF64(t *testing.T) { func runSyncUpdateInsertShuffle2FilesWithSmallCap[T types.RealNumbers](t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) proc.SetResolveVariableFunc(func(key string, b1 bool, b2 bool) (any, error) { switch key { @@ -548,14 +588,83 @@ func runSyncUpdateInsertShuffle2FilesWithSmallCap[T types.RealNumbers](t *testin var ff T switch any(ff).(type) { case float32: - err := CdcSync[T](proc, "db", "src", int32(types.T_array_float32), 3, &cdc) + sync, err := NewHnswSync[T](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) require.Nil(t, err) case float64: - err := CdcSync[T](proc, "db", "src", int32(types.T_array_float64), 3, &cdc) + sync, err := NewHnswSync[T](sqlproc, "db", "src", int32(types.T_array_float64), 3) require.Nil(t, err) + err = sync.RunOnce(sqlproc, &cdc) + require.Nil(t, err) + } } func TestSyncUpdateInsertShuffle2FilesF32WithSmallCap(t *testing.T) { runSyncUpdateInsertShuffle2FilesWithSmallCap[float32](t) } + +func runSyncContinuousUpdateInsertShuffle2FilesWithSmallCap[T types.RealNumbers](t *testing.T) { + m := mpool.MustNewZero() + proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) + + proc.SetResolveVariableFunc(func(key string, b1 bool, b2 bool) (any, error) { + switch key { + case "hnsw_max_index_capacity": + return int64(10), nil + case "hnsw_threads_build": + return int64(8), nil + default: + return int64(0), nil + } + }) + + runSql = mock_runSql_2files + runSql_streaming = mock_runSql_streaming_2files + runCatalogSql = mock_runCatalogSql + runTxn = mock_runTxn + + cdc := vectorindex.VectorIndexCdc[T]{Data: make([]vectorindex.VectorIndexCdcEntry[T], 0, 100)} + + key := int64(0) + v := []T{0.1, 0.2, 0.3} + + // 0 - 199 key exists, 200 - 399 new insert + for i := 0; i < 400; i++ { + e := vectorindex.VectorIndexCdcEntry[T]{Type: vectorindex.CDC_UPSERT, PKey: key, Vec: v} + cdc.Data = append(cdc.Data, e) + key += 1 + } + + rand.Seed(uint64(time.Now().UnixNano())) + rand.Shuffle(len(cdc.Data), func(i, j int) { cdc.Data[i], cdc.Data[j] = cdc.Data[j], cdc.Data[i] }) + + var err error + var sync *HnswSync[T] + + var ff T + switch any(ff).(type) { + case float32: + sync, err = NewHnswSync[T](sqlproc, "db", "src", int32(types.T_array_float32), 3) + require.Nil(t, err) + + case float64: + sync, err = NewHnswSync[T](sqlproc, "db", "src", int32(types.T_array_float64), 3) + require.Nil(t, err) + } + + defer sync.Destroy() + for i := 0; i < 10; i++ { + err = sync.Update(sqlproc, &cdc) + require.Nil(t, err) + } + + err = sync.Save(sqlproc) + require.NoError(t, err) +} + +func TestSyncContinuousUpdateInsertShuffle2FilesF32WithSmallCap(t *testing.T) { + runSyncContinuousUpdateInsertShuffle2FilesWithSmallCap[float32](t) +} diff --git a/pkg/vectorindex/ivfflat/search.go b/pkg/vectorindex/ivfflat/search.go index 8afef09dd872d..171a6ae81a903 100644 --- a/pkg/vectorindex/ivfflat/search.go +++ b/pkg/vectorindex/ivfflat/search.go @@ -31,7 +31,6 @@ import ( "github.com/matrixorigin/matrixone/pkg/vectorindex/cache" "github.com/matrixorigin/matrixone/pkg/vectorindex/metric" "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" - "github.com/matrixorigin/matrixone/pkg/vm/process" ) var runSql = sqlexec.RunSql @@ -56,7 +55,7 @@ type IvfflatSearch[T types.RealNumbers] struct { ThreadsSearch int64 } -func (idx *IvfflatSearchIndex[T]) LoadIndex(proc *process.Process, idxcfg vectorindex.IndexConfig, tblcfg vectorindex.IndexTableConfig, nthread int64) error { +func (idx *IvfflatSearchIndex[T]) LoadIndex(proc *sqlexec.SqlProcess, idxcfg vectorindex.IndexConfig, tblcfg vectorindex.IndexTableConfig, nthread int64) error { idx.Version = idxcfg.Ivfflat.Version sql := fmt.Sprintf( @@ -103,7 +102,7 @@ func (idx *IvfflatSearchIndex[T]) LoadIndex(proc *process.Process, idxcfg vector // load chunk from database func (idx *IvfflatSearchIndex[T]) searchEntries( ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, query []T, distfn metric.DistanceFunction[T], heap *vectorindex.SearchResultSafeHeap, @@ -116,6 +115,8 @@ func (idx *IvfflatSearchIndex[T]) searchEntries( res executor.Result ) + procCtx := sqlproc.GetContext() + select { case res, ok = <-stream_chan: if !ok { @@ -123,8 +124,8 @@ func (idx *IvfflatSearchIndex[T]) searchEntries( } case err = <-error_chan: return false, err - case <-proc.Ctx.Done(): - return false, proc.Ctx.Err() + case <-procCtx.Done(): + return false, procCtx.Err() case <-ctx.Done(): // local context cancelled. something went wrong with other threads return false, context.Cause(ctx) @@ -149,7 +150,7 @@ func (idx *IvfflatSearchIndex[T]) searchEntries( return false, nil } -func (idx *IvfflatSearchIndex[T]) findCentroids(proc *process.Process, query []T, distfn metric.DistanceFunction[T], idxcfg vectorindex.IndexConfig, probe uint, nthread int64) ([]int64, error) { +func (idx *IvfflatSearchIndex[T]) findCentroids(sqlproc *sqlexec.SqlProcess, query []T, distfn metric.DistanceFunction[T], idxcfg vectorindex.IndexConfig, probe uint, nthread int64) ([]int64, error) { if len(idx.Centroids) == 0 { // empty index has id = 1 @@ -195,7 +196,7 @@ func (idx *IvfflatSearchIndex[T]) findCentroids(proc *process.Process, query []T srif := heap.Pop() sr, ok := srif.(*vectorindex.SearchResult) if !ok { - return nil, moerr.NewInternalError(proc.Ctx, "findCentroids: heap return key is not int64") + return nil, moerr.NewInternalError(sqlproc.GetContext(), "findCentroids: heap return key is not int64") } res = append(res, sr.Id) } @@ -206,7 +207,7 @@ func (idx *IvfflatSearchIndex[T]) findCentroids(proc *process.Process, query []T // Call usearch.Search func (idx *IvfflatSearchIndex[T]) Search( - proc *process.Process, + sqlproc *sqlexec.SqlProcess, idxcfg vectorindex.IndexConfig, tblcfg vectorindex.IndexTableConfig, query []T, @@ -222,7 +223,7 @@ func (idx *IvfflatSearchIndex[T]) Search( return nil, nil, err } - centroids_ids, err := idx.findCentroids(proc, query, distfn, idxcfg, rt.Probe, nthread) + centroids_ids, err := idx.findCentroids(sqlproc, query, distfn, idxcfg, rt.Probe, nthread) if err != nil { return nil, nil, err } @@ -249,7 +250,7 @@ func (idx *IvfflatSearchIndex[T]) Search( var ( wg sync.WaitGroup - ctx, cancel = context.WithCancelCause(proc.GetTopContext()) + ctx, cancel = context.WithCancelCause(sqlproc.GetTopContext()) ) defer cancel(nil) @@ -257,7 +258,7 @@ func (idx *IvfflatSearchIndex[T]) Search( go func() { defer wg.Done() if _, err2 := runSql_streaming( - ctx, proc, sql, stream_chan, error_chan, + ctx, sqlproc, sql, stream_chan, error_chan, ); err2 != nil { // consumer notify the producer to stop the sql streaming cancel(err2) @@ -282,7 +283,7 @@ func (idx *IvfflatSearchIndex[T]) Search( ) for !streamClosed { if streamClosed, err2 = idx.searchEntries( - ctx, proc, query, distfn, heap, stream_chan, error_chan, + ctx, sqlproc, query, distfn, heap, stream_chan, error_chan, ); err2 != nil { // consumer notify the producer to stop the sql streaming cancel(err2) @@ -329,10 +330,12 @@ func (idx *IvfflatSearchIndex[T]) Search( default: } + procCtx := sqlproc.GetContext() + // check local context cancelled select { - case <-proc.Ctx.Done(): - err = proc.Ctx.Err() + case <-procCtx.Done(): + err = procCtx.Err() return case <-ctx.Done(): err = context.Cause(ctx) @@ -349,7 +352,7 @@ func (idx *IvfflatSearchIndex[T]) Search( srif := heap.Pop() sr, ok := srif.(*vectorindex.SearchResultAnyKey) if !ok { - err = moerr.NewInternalError(proc.Ctx, "ivf search: heap return key is not any") + err = moerr.NewInternalError(sqlproc.GetContext(), "ivf search: heap return key is not any") return } resid = append(resid, sr.Id) @@ -374,14 +377,14 @@ func NewIvfflatSearch[T types.RealNumbers]( // Search the hnsw index (implement VectorIndexSearch.Search) func (s *IvfflatSearch[T]) Search( - proc *process.Process, anyquery any, rt vectorindex.RuntimeConfig, + sqlproc *sqlexec.SqlProcess, anyquery any, rt vectorindex.RuntimeConfig, ) (keys any, distances []float64, err error) { query, ok := anyquery.([]T) if !ok { return nil, nil, moerr.NewInternalErrorNoCtx("IvfSearch: query not match with index type") } - return s.Index.Search(proc, s.Idxcfg, s.Tblcfg, query, rt, s.ThreadsSearch) + return s.Index.Search(sqlproc, s.Idxcfg, s.Tblcfg, query, rt, s.ThreadsSearch) } func (s *IvfflatSearch[T]) Contains(key int64) (bool, error) { @@ -398,11 +401,11 @@ func (s *IvfflatSearch[T]) Destroy() { } // load index from database (implement VectorIndexSearch.LoadFromDatabase) -func (s *IvfflatSearch[T]) Load(proc *process.Process) error { +func (s *IvfflatSearch[T]) Load(sqlproc *sqlexec.SqlProcess) error { idx := &IvfflatSearchIndex[T]{} // load index model - err := idx.LoadIndex(proc, s.Idxcfg, s.Tblcfg, s.ThreadsSearch) + err := idx.LoadIndex(sqlproc, s.Idxcfg, s.Tblcfg, s.ThreadsSearch) if err != nil { return err } diff --git a/pkg/vectorindex/ivfflat/search_test.go b/pkg/vectorindex/ivfflat/search_test.go index 20c677992aebe..8ace372ac619b 100644 --- a/pkg/vectorindex/ivfflat/search_test.go +++ b/pkg/vectorindex/ivfflat/search_test.go @@ -24,14 +24,14 @@ import ( "github.com/matrixorigin/matrixone/pkg/util/executor" "github.com/matrixorigin/matrixone/pkg/vectorindex" "github.com/matrixorigin/matrixone/pkg/vectorindex/metric" - "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" "github.com/stretchr/testify/require" ) // give blob func mock_runSql_streaming( ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, sql string, ch chan executor.Result, err_chan chan error, @@ -44,7 +44,7 @@ func mock_runSql_streaming( func mock_runSql_streaming_parser_error( ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, sql string, ch chan executor.Result, err_chan chan error, @@ -55,11 +55,13 @@ func mock_runSql_streaming_parser_error( func mock_runSql_streaming_cancel( ctx context.Context, - proc *process.Process, + sqlproc *sqlexec.SqlProcess, sql string, ch chan executor.Result, err_chan chan error, ) (executor.Result, error) { + + proc := sqlproc.Proc select { case <-proc.Ctx.Done(): close(ch) @@ -81,6 +83,7 @@ func TestIvfSearchRace(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) idxcfg.Ivfflat.Metric = uint16(metric.Metric_L2Distance) @@ -89,7 +92,7 @@ func TestIvfSearchRace(t *testing.T) { idx := &IvfflatSearchIndex[float32]{} - _, _, err := idx.Search(proc, idxcfg, tblcfg, v, rt, 4) + _, _, err := idx.Search(sqlproc, idxcfg, tblcfg, v, rt, 4) require.NotNil(t, err) } @@ -103,6 +106,7 @@ func TestIvfSearchParserError(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := sqlexec.NewSqlProcess(proc) idxcfg.Ivfflat.Metric = uint16(metric.Metric_L2Distance) @@ -111,7 +115,7 @@ func TestIvfSearchParserError(t *testing.T) { idx := &IvfflatSearchIndex[float32]{} - _, _, err := idx.Search(proc, idxcfg, tblcfg, v, rt, 4) + _, _, err := idx.Search(sqlproc, idxcfg, tblcfg, v, rt, 4) require.NotNil(t, err) } @@ -125,6 +129,7 @@ func TestIvfSearchCancel(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) proc.Ctx, proc.Cancel = context.WithCancelCause(proc.Ctx) + sqlproc := sqlexec.NewSqlProcess(proc) idxcfg.Ivfflat.Metric = uint16(metric.Metric_L2Distance) @@ -135,7 +140,7 @@ func TestIvfSearchCancel(t *testing.T) { proc.Cancel(moerr.NewInternalErrorNoCtx("user cancel")) - _, _, err := idx.Search(proc, idxcfg, tblcfg, v, rt, 4) + _, _, err := idx.Search(sqlproc, idxcfg, tblcfg, v, rt, 4) t.Logf("error: %v", err) require.Error(t, err) } diff --git a/pkg/vectorindex/ivfflat/sql.go b/pkg/vectorindex/ivfflat/sql.go index 62136060d32cb..0e38bce5d8899 100644 --- a/pkg/vectorindex/ivfflat/sql.go +++ b/pkg/vectorindex/ivfflat/sql.go @@ -21,24 +21,24 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/vectorindex" - "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/matrixorigin/matrixone/pkg/vectorindex/sqlexec" ) -func GetVersion(proc *process.Process, tblcfg vectorindex.IndexTableConfig) (int64, error) { +func GetVersion(sqlproc *sqlexec.SqlProcess, tblcfg vectorindex.IndexTableConfig) (int64, error) { sql := fmt.Sprintf("SELECT CAST(`%s` AS BIGINT) FROM `%s`.`%s` WHERE `%s` = 'version'", catalog.SystemSI_IVFFLAT_TblCol_Metadata_val, tblcfg.DbName, tblcfg.MetadataTable, catalog.SystemSI_IVFFLAT_TblCol_Metadata_key) - res, err := runSql(proc, sql) + res, err := runSql(sqlproc, sql) if err != nil { return 0, err } defer res.Close() if len(res.Batches) == 0 { - return 0, moerr.NewInternalError(proc.Ctx, "version not found") + return 0, moerr.NewInternalError(sqlproc.GetContext(), "version not found") } version := int64(0) diff --git a/pkg/vectorindex/sqlexec/sqlexec.go b/pkg/vectorindex/sqlexec/sqlexec.go index f0c4e69c54ea0..b2db8f68f1be6 100644 --- a/pkg/vectorindex/sqlexec/sqlexec.go +++ b/pkg/vectorindex/sqlexec/sqlexec.go @@ -16,95 +16,307 @@ package sqlexec import ( "context" + "errors" + "time" moruntime "github.com/matrixorigin/matrixone/pkg/common/runtime" "github.com/matrixorigin/matrixone/pkg/defines" + "github.com/matrixorigin/matrixone/pkg/txn/client" "github.com/matrixorigin/matrixone/pkg/util/executor" + "github.com/matrixorigin/matrixone/pkg/vm/engine" "github.com/matrixorigin/matrixone/pkg/vm/process" ) -// run SQL in batch mode. Result batches will stored in memory and return once all result batches received. -func RunSql(proc *process.Process, sql string) (executor.Result, error) { - v, ok := moruntime.ServiceRuntime(proc.GetService()).GetGlobalVariables(moruntime.InternalSQLExecutor) - if !ok { - panic("missing lock service") +// SqlContext stores required information for background SQLInternalExecutor +type SqlContext struct { + Ctx context.Context + CNUuid string + TxnOperator client.TxnOperator + AccountId uint32 + ResolveVariableFunc func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error) +} + +func NewSqlContext(ctx context.Context, + cnuuid string, + txnOperator client.TxnOperator, + accountid uint32, + resolveVariableFunc func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error), +) *SqlContext { + return &SqlContext{Ctx: ctx, CNUuid: cnuuid, TxnOperator: txnOperator, AccountId: accountid, ResolveVariableFunc: resolveVariableFunc} +} + +func (s *SqlContext) GetService() string { + return s.CNUuid +} + +func (s *SqlContext) Txn() client.TxnOperator { + return s.TxnOperator +} + +func (s *SqlContext) GetResolveVariableFunc() func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error) { + return s.ResolveVariableFunc +} + +func (s *SqlContext) SetResolveVariableFunc(f func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error)) { + s.ResolveVariableFunc = f +} + +// SqlProcess is the wrapper for both process.Process and background SQLContext +// SqlProcess enable the API to run in both frontend and background with InternalSQLExecutor +// process.Process always exists in frontend. +// However, process.Process does not exist in background job. +// SqlContext with required infos such as context.Context, CNUUID, TxnOperator and AccountId enable +// to run SQL with InternalSQLExecutor. +// Either process.Process or SqlContext is used in SqlProcess. +// We will look for process.Process first before SqlContext +type SqlProcess struct { + Proc *process.Process + SqlCtx *SqlContext +} + +func NewSqlProcess(proc *process.Process) *SqlProcess { + return &SqlProcess{Proc: proc} +} + +func NewSqlProcessWithContext(ctx *SqlContext) *SqlProcess { + return &SqlProcess{SqlCtx: ctx} +} + +func (s *SqlProcess) GetContext() context.Context { + if s.Proc != nil { + return s.Proc.Ctx + } + return s.SqlCtx.Ctx +} + +func (s *SqlProcess) GetTopContext() context.Context { + if s.Proc != nil { + return s.Proc.GetTopContext() } + return s.SqlCtx.Ctx +} - //------------------------------------------------------- - topContext := proc.GetTopContext() - accountId, err := defines.GetAccountId(proc.Ctx) - if err != nil { - return executor.Result{}, err +func (s *SqlProcess) GetResolveVariableFunc() func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error) { + if s.Proc != nil { + return s.Proc.GetResolveVariableFunc() } - //------------------------------------------------------- + if s.SqlCtx != nil { + return s.SqlCtx.GetResolveVariableFunc() + } + return nil +} + +// run SQL in batch mode. Result batches will stored in memory and return once all result batches received. +func RunSql(sqlproc *SqlProcess, sql string) (executor.Result, error) { + if sqlproc.Proc != nil { + proc := sqlproc.Proc + v, ok := moruntime.ServiceRuntime(proc.GetService()).GetGlobalVariables(moruntime.InternalSQLExecutor) + if !ok { + panic("missing lock service") + } + + //------------------------------------------------------- + topContext := proc.GetTopContext() + accountId, err := defines.GetAccountId(proc.Ctx) + if err != nil { + return executor.Result{}, err + } + //------------------------------------------------------- + + exec := v.(executor.SQLExecutor) + opts := executor.Options{}. + // All runSql and runSqlWithResult is a part of input sql, can not incr statement. + // All these sub-sql's need to be rolled back and retried en masse when they conflict in pessimistic mode + WithDisableIncrStatement(). + WithTxn(proc.GetTxnOperator()). + WithDatabase(proc.GetSessionInfo().Database). + WithTimeZone(proc.GetSessionInfo().TimeZone). + WithAccountID(accountId). + WithResolveVariableFunc(proc.GetResolveVariableFunc()) + return exec.Exec(topContext, sql, opts) + } else { + + sqlctx := sqlproc.SqlCtx + v, ok := moruntime.ServiceRuntime(sqlctx.GetService()).GetGlobalVariables(moruntime.InternalSQLExecutor) + if !ok { + panic("missing lock service") + } - exec := v.(executor.SQLExecutor) - opts := executor.Options{}. - // All runSql and runSqlWithResult is a part of input sql, can not incr statement. - // All these sub-sql's need to be rolled back and retried en masse when they conflict in pessimistic mode - WithDisableIncrStatement(). - WithTxn(proc.GetTxnOperator()). - WithDatabase(proc.GetSessionInfo().Database). - WithTimeZone(proc.GetSessionInfo().TimeZone). - WithAccountID(accountId) - return exec.Exec(topContext, sql, opts) + accountId := sqlctx.AccountId + + exec := v.(executor.SQLExecutor) + opts := executor.Options{}. + // All runSql and runSqlWithResult is a part of input sql, can not incr statement. + // All these sub-sql's need to be rolled back and retried en masse when they conflict in pessimistic mode + WithDisableIncrStatement(). + WithTxn(sqlctx.Txn()). + WithAccountID(accountId). + WithResolveVariableFunc(sqlctx.GetResolveVariableFunc()) + return exec.Exec(sqlctx.Ctx, sql, opts) + + } } // run SQL in WithStreaming() and pass the channel to SQL executor func RunStreamingSql( ctx context.Context, - proc *process.Process, + sqlproc *SqlProcess, sql string, stream_chan chan executor.Result, error_chan chan error, ) (executor.Result, error) { - v, ok := moruntime.ServiceRuntime(proc.GetService()).GetGlobalVariables(moruntime.InternalSQLExecutor) - if !ok { - panic("missing lock service") - } - //------------------------------------------------------- - accountId, err := defines.GetAccountId(proc.Ctx) - if err != nil { - return executor.Result{}, err + if sqlproc.Proc != nil { + proc := sqlproc.Proc + v, ok := moruntime.ServiceRuntime(proc.GetService()).GetGlobalVariables(moruntime.InternalSQLExecutor) + if !ok { + panic("missing lock service") + } + + //------------------------------------------------------- + accountId, err := defines.GetAccountId(proc.Ctx) + if err != nil { + return executor.Result{}, err + } + //------------------------------------------------------- + exec := v.(executor.SQLExecutor) + opts := executor.Options{}. + // All runSql and runSqlWithResult is a part of input sql, can not incr statement. + // All these sub-sql's need to be rolled back and retried en masse when they conflict in pessimistic mode + WithDisableIncrStatement(). + WithTxn(proc.GetTxnOperator()). + WithDatabase(proc.GetSessionInfo().Database). + WithTimeZone(proc.GetSessionInfo().TimeZone). + WithAccountID(accountId). + WithStreaming(stream_chan, error_chan). + WithResolveVariableFunc(proc.GetResolveVariableFunc()) + return exec.Exec(ctx, sql, opts) + } else { + + sqlctx := sqlproc.SqlCtx + + v, ok := moruntime.ServiceRuntime(sqlctx.GetService()).GetGlobalVariables(moruntime.InternalSQLExecutor) + if !ok { + panic("missing lock service") + } + + accountId := sqlctx.AccountId + + exec := v.(executor.SQLExecutor) + opts := executor.Options{}. + // All runSql and runSqlWithResult is a part of input sql, can not incr statement. + // All these sub-sql's need to be rolled back and retried en masse when they conflict in pessimistic mode + WithDisableIncrStatement(). + WithTxn(sqlctx.Txn()). + WithAccountID(accountId). + WithStreaming(stream_chan, error_chan). + WithResolveVariableFunc(sqlctx.GetResolveVariableFunc()) + return exec.Exec(ctx, sql, opts) + } - //------------------------------------------------------- - exec := v.(executor.SQLExecutor) - opts := executor.Options{}. - // All runSql and runSqlWithResult is a part of input sql, can not incr statement. - // All these sub-sql's need to be rolled back and retried en masse when they conflict in pessimistic mode - WithDisableIncrStatement(). - WithTxn(proc.GetTxnOperator()). - WithDatabase(proc.GetSessionInfo().Database). - WithTimeZone(proc.GetSessionInfo().TimeZone). - WithAccountID(accountId). - WithStreaming(stream_chan, error_chan) - return exec.Exec(ctx, sql, opts) + } // run SQL in batch mode. Result batches will stored in memory and return once all result batches received. -func RunTxn(proc *process.Process, execFunc func(executor.TxnExecutor) error) error { - v, ok := moruntime.ServiceRuntime(proc.GetService()).GetGlobalVariables(moruntime.InternalSQLExecutor) - if !ok { - panic("missing lock service") +func RunTxn(sqlproc *SqlProcess, execFunc func(executor.TxnExecutor) error) error { + if sqlproc.Proc != nil { + proc := sqlproc.Proc + + v, ok := moruntime.ServiceRuntime(proc.GetService()).GetGlobalVariables(moruntime.InternalSQLExecutor) + if !ok { + panic("missing lock service") + } + + //------------------------------------------------------- + topContext := proc.GetTopContext() + accountId, err := defines.GetAccountId(proc.Ctx) + if err != nil { + return err + } + //------------------------------------------------------- + + exec := v.(executor.SQLExecutor) + opts := executor.Options{}. + // All runSql and runSqlWithResult is a part of input sql, can not incr statement. + // All these sub-sql's need to be rolled back and retried en masse when they conflict in pessimistic mode + WithDisableIncrStatement(). + WithTxn(proc.GetTxnOperator()). + WithDatabase(proc.GetSessionInfo().Database). + WithTimeZone(proc.GetSessionInfo().TimeZone). + WithAccountID(accountId). + WithResolveVariableFunc(proc.GetResolveVariableFunc()) + return exec.ExecTxn(topContext, execFunc, opts) + } else { + + sqlctx := sqlproc.SqlCtx + v, ok := moruntime.ServiceRuntime(sqlctx.GetService()).GetGlobalVariables(moruntime.InternalSQLExecutor) + if !ok { + panic("missing lock service") + } + + accountId := sqlctx.AccountId + + exec := v.(executor.SQLExecutor) + opts := executor.Options{}. + // All runSql and runSqlWithResult is a part of input sql, can not incr statement. + // All these sub-sql's need to be rolled back and retried en masse when they conflict in pessimistic mode + WithDisableIncrStatement(). + WithTxn(sqlctx.Txn()). + WithAccountID(accountId). + WithResolveVariableFunc(sqlctx.GetResolveVariableFunc()) + return exec.ExecTxn(sqlctx.Ctx, execFunc, opts) } +} - //------------------------------------------------------- - topContext := proc.GetTopContext() - accountId, err := defines.GetAccountId(proc.Ctx) +func getTxn( + ctx context.Context, + cnEngine engine.Engine, + cnTxnClient client.TxnClient, + info string, +) (client.TxnOperator, error) { + nowTs := cnEngine.LatestLogtailAppliedTime() + createByOpt := client.WithTxnCreateBy( + 0, + "", + info, + 0) + op, err := cnTxnClient.New(ctx, nowTs, createByOpt) + if err != nil { + return nil, err + } + err = cnEngine.New(ctx, op) + if err != nil { + return nil, err + } + return op, nil +} + +// run SQL with SqlContext +func RunTxnWithSqlContext(ctx context.Context, + cnEngine engine.Engine, + cnTxnClient client.TxnClient, + cnUUID string, + accountId uint32, + duration time.Duration, + resolveVariableFunc func(varName string, isSystemVar, isGlobalVar bool) (interface{}, error), + cbdata any, + f func(sqlproc *SqlProcess, data any) error) (err error) { + + newctx := context.WithValue(context.Background(), defines.TenantIDKey{}, accountId) + newctx, cancel := context.WithTimeout(newctx, duration) + defer cancel() + + txnOp, err := getTxn(newctx, cnEngine, cnTxnClient, "runTxnWithSqlContext") if err != nil { return err } - //------------------------------------------------------- - - exec := v.(executor.SQLExecutor) - opts := executor.Options{}. - // All runSql and runSqlWithResult is a part of input sql, can not incr statement. - // All these sub-sql's need to be rolled back and retried en masse when they conflict in pessimistic mode - WithDisableIncrStatement(). - WithTxn(proc.GetTxnOperator()). - WithDatabase(proc.GetSessionInfo().Database). - WithTimeZone(proc.GetSessionInfo().TimeZone). - WithAccountID(accountId) - return exec.ExecTxn(topContext, execFunc, opts) + + sqlproc := NewSqlProcessWithContext(NewSqlContext(newctx, cnUUID, txnOp, accountId, resolveVariableFunc)) + err = f(sqlproc, cbdata) + if err != nil { + err = errors.Join(err, txnOp.Rollback(sqlproc.GetContext())) + } else { + err = txnOp.Commit(sqlproc.GetContext()) + } + return } diff --git a/pkg/vectorindex/sqlexec/sqlexec_test.go b/pkg/vectorindex/sqlexec/sqlexec_test.go index 57dd13f330444..f0fe01ab3a3b0 100644 --- a/pkg/vectorindex/sqlexec/sqlexec_test.go +++ b/pkg/vectorindex/sqlexec/sqlexec_test.go @@ -46,8 +46,9 @@ func TestSqlTxnError(t *testing.T) { m := mpool.MustNewZero() proc := testutil.NewProcessWithMPool(t, "", m) + sqlproc := NewSqlProcess(proc) assert.Panics(t, func() { - RunTxn(proc, func(exec executor.TxnExecutor) error { + RunTxn(sqlproc, func(exec executor.TxnExecutor) error { return nil }) }, "logserivce panic") @@ -66,7 +67,9 @@ func TestSqlTxn(t *testing.T) { proc.Ctx = context.Background() proc.Ctx = context.WithValue(proc.Ctx, defines.TenantIDKey{}, uint32(0)) - err := RunTxn(proc, func(exec executor.TxnExecutor) error { + sqlproc := NewSqlProcess(proc) + + err := RunTxn(sqlproc, func(exec executor.TxnExecutor) error { return nil }) require.Nil(t, err) diff --git a/pkg/vectorindex/types.go b/pkg/vectorindex/types.go index 5fcd032b1f021..c2ea6f65401a1 100644 --- a/pkg/vectorindex/types.go +++ b/pkg/vectorindex/types.go @@ -15,9 +15,9 @@ package vectorindex import ( - "encoding/json" "runtime" + "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/container/types" usearch "github.com/unum-cloud/usearch/golang" ) @@ -112,9 +112,9 @@ type VectorIndexCdc[T types.RealNumbers] struct { Data []VectorIndexCdcEntry[T] `json:"cdc"` } -func NewVectorIndexCdc[T types.RealNumbers]() *VectorIndexCdc[T] { +func NewVectorIndexCdc[T types.RealNumbers](capacity int) *VectorIndexCdc[T] { return &VectorIndexCdc[T]{ - Data: make([]VectorIndexCdcEntry[T], 0, 8192), + Data: make([]VectorIndexCdcEntry[T], 0, capacity), } } @@ -161,7 +161,7 @@ func (h *VectorIndexCdc[T]) Delete(key int64) { func (h *VectorIndexCdc[T]) ToJson() (string, error) { - b, err := json.Marshal(h) + b, err := sonic.Marshal(h) if err != nil { return "", err } diff --git a/pkg/vectorindex/types_test.go b/pkg/vectorindex/types_test.go index fad882167d8af..445cd9fd0732d 100644 --- a/pkg/vectorindex/types_test.go +++ b/pkg/vectorindex/types_test.go @@ -26,7 +26,7 @@ func TestCdc(t *testing.T) { key2 := int64(1) v2 := []float32{1, 2, 3} - cdc := NewVectorIndexCdc[float32]() + cdc := NewVectorIndexCdc[float32](8192) // Insert cdc.Insert(key, v) diff --git a/pkg/vm/engine/entire_engine_test.go b/pkg/vm/engine/entire_engine_test.go index 2b61c328ddafb..0912c402d0125 100644 --- a/pkg/vm/engine/entire_engine_test.go +++ b/pkg/vm/engine/entire_engine_test.go @@ -461,7 +461,7 @@ func (o *testOperator) IsOpenLog() bool { panic("unimplemented") } -func (o *testOperator) AppendEventCallback(event client.EventType, callbacks ...func(event client.TxnEvent)) { +func (o *testOperator) AppendEventCallback(event client.EventType, callbacks ...client.TxnEventCallback) { panic("unimplemented") } diff --git a/test/distributed/cases/fulltext/fulltext_async.result b/test/distributed/cases/fulltext/fulltext_async.result new file mode 100644 index 0000000000000..bc2e78807b937 --- /dev/null +++ b/test/distributed/cases/fulltext/fulltext_async.result @@ -0,0 +1,23 @@ +set experimental_fulltext_index=1; +set ft_relevancy_algorithm="TF-IDF"; +create table src (id bigint primary key, body varchar, title text, FULLTEXT ftidx (body, title) ASYNC); +insert into src values (0, 'color is red', 't1'), (1, 'car is yellow', 'crazy car'), (2, 'sky is blue', 'no limit'), (3, 'blue is not red', 'colorful'), +(4, '遠東兒童中文是針對6到9歲的小朋友精心設計的中文學習教材,共三冊,目前已出版一、二冊。', '遠東兒童中文'), +(5, '每冊均採用近百張全幅彩圖及照片,生動活潑、自然真實,加深兒童學習印象,洋溢學習樂趣。', '遠東兒童中文'), +(6, '各個單元主題內容涵蓋中華文化及生活應用的介紹。本套教材含課本、教學指引、生字卡、學生作業本與CD,中英對照,精美大字版。本系列有繁體字及簡體字兩種版本印行。', '中文短篇小說'), +(7, '59個簡單的英文和中文短篇小說', '適合初學者'), +(8, NULL, 'NOT INCLUDED'), +(9, 'NOT INCLUDED BODY', NULL), +(10, NULL, NULL); +select sleep(30); +sleep(30) +0 +select * from src where match(body, title) against('red'); +id body title +0 color is red t1 +3 blue is not red colorful +show create table src; +Table Create Table +src CREATE TABLE `src` (\n `id` bigint NOT NULL,\n `body` varchar(65535) DEFAULT NULL,\n `title` text DEFAULT NULL,\n PRIMARY KEY (`id`),\n FULLTEXT `ftidx`(`body`,`title`) ASYNC\n) +alter table src rename to src1; +drop table src1; diff --git a/test/distributed/cases/fulltext/fulltext_async.sql b/test/distributed/cases/fulltext/fulltext_async.sql new file mode 100644 index 0000000000000..c982c871679f8 --- /dev/null +++ b/test/distributed/cases/fulltext/fulltext_async.sql @@ -0,0 +1,25 @@ +-- TODO: run all tests with both experimental_fulltext_index = 0 and 1 +-- TODO: GENERATE the test case to cover all combinations of types (varchar, char and text) +set experimental_fulltext_index=1; +set ft_relevancy_algorithm="TF-IDF"; + +create table src (id bigint primary key, body varchar, title text, FULLTEXT ftidx (body, title) ASYNC); + +insert into src values (0, 'color is red', 't1'), (1, 'car is yellow', 'crazy car'), (2, 'sky is blue', 'no limit'), (3, 'blue is not red', 'colorful'), +(4, '遠東兒童中文是針對6到9歲的小朋友精心設計的中文學習教材,共三冊,目前已出版一、二冊。', '遠東兒童中文'), +(5, '每冊均採用近百張全幅彩圖及照片,生動活潑、自然真實,加深兒童學習印象,洋溢學習樂趣。', '遠東兒童中文'), +(6, '各個單元主題內容涵蓋中華文化及生活應用的介紹。本套教材含課本、教學指引、生字卡、學生作業本與CD,中英對照,精美大字版。本系列有繁體字及簡體字兩種版本印行。', '中文短篇小說'), +(7, '59個簡單的英文和中文短篇小說', '適合初學者'), +(8, NULL, 'NOT INCLUDED'), +(9, 'NOT INCLUDED BODY', NULL), +(10, NULL, NULL); + +select sleep(30); + +select * from src where match(body, title) against('red'); + +show create table src; + +alter table src rename to src1; + +drop table src1; diff --git a/test/distributed/cases/vector/vector_hnsw.result b/test/distributed/cases/vector/vector_hnsw.result index 2e2822602b8ef..1ff4c9b888db1 100644 --- a/test/distributed/cases/vector/vector_hnsw.result +++ b/test/distributed/cases/vector/vector_hnsw.result @@ -16,10 +16,18 @@ Field Type Null Key Default Extra Comment a BIGINT(64) NO PRI null b VECF32(128) YES MUL null c INT(32) YES MUL null +select sleep(30); +sleep(30) +0 select * from vector_index_01 order by L2_DISTANCE(b, "[16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") ASC LIMIT 2; a b c +9777 [16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13] 4 +9776 [10, 3, 8, 5, 48, 26, 5, 16, 17, 0, 0, 2, 132, 53, 1, 16, 112, 6, 0, 0, 7, 2, 1, 48, 48, 15, 18, 31, 3, 0, 0, 9, 6, 10, 19, 27, 50, 46, 17, 9, 18, 1, 4, 48, 132, 23, 3, 5, 132, 9, 4, 3, 11, 0, 2, 46, 84, 12, 10, 10, 1, 0, 12, 76, 26, 22, 16, 26, 35, 15, 3, 16, 15, 1, 51, 132, 125, 8, 1, 2, 132, 51, 67, 91, 8, 0, 0, 30, 126, 39, 32, 38, 4, 0, 1, 12, 24, 2, 2, 2, 4, 7, 2, 19, 93, 19, 70, 92, 2, 3, 1, 21, 36, 58, 132, 94, 0, 0, 0, 0, 21, 25, 57, 48, 1, 0, 0, 1] 3 select * from vector_index_01 where a>9774 order by L2_DISTANCE(b, "[4, 6, 1, 42, 119, 4, 1, 0, 9, 96, 58, 2, 14, 0, 0, 0, 0, 57, 119, 24, 15, 2, 0, 0, 0, 0, 48, 26, 26, 7, 0, 0, 66, 21, 0, 4, 107, 92, 42, 7, 119, 119, 45, 2, 21, 18, 0, 9, 0, 39, 67, 43, 101, 66, 0, 0, 0, 3, 38, 75, 76, 17, 0, 0, 89, 0, 0, 0, 6, 26, 21, 20, 119, 1, 0, 0, 10, 43, 72, 92, 2, 8, 22, 25, 22, 46, 119, 60, 1, 13, 45, 48, 75, 69, 45, 15, 13, 0, 0, 2, 6, 0, 1, 11, 24, 0, 0, 0, 0, 24, 92, 49, 0, 0, 0, 0, 0, 18, 119, 40, 0, 0, 0, 0, 4, 47, 81, 10]") desc limit 2; a b c +9776 [10, 3, 8, 5, 48, 26, 5, 16, 17, 0, 0, 2, 132, 53, 1, 16, 112, 6, 0, 0, 7, 2, 1, 48, 48, 15, 18, 31, 3, 0, 0, 9, 6, 10, 19, 27, 50, 46, 17, 9, 18, 1, 4, 48, 132, 23, 3, 5, 132, 9, 4, 3, 11, 0, 2, 46, 84, 12, 10, 10, 1, 0, 12, 76, 26, 22, 16, 26, 35, 15, 3, 16, 15, 1, 51, 132, 125, 8, 1, 2, 132, 51, 67, 91, 8, 0, 0, 30, 126, 39, 32, 38, 4, 0, 1, 12, 24, 2, 2, 2, 4, 7, 2, 19, 93, 19, 70, 92, 2, 3, 1, 21, 36, 58, 132, 94, 0, 0, 0, 0, 21, 25, 57, 48, 1, 0, 0, 1] 3 +9775 [0, 1, 1, 3, 0, 3, 46, 20, 1, 4, 17, 9, 1, 17, 108, 15, 0, 3, 37, 17, 6, 15, 116, 16, 6, 1, 4, 7, 7, 7, 9, 6, 0, 8, 10, 4, 26, 129, 27, 9, 0, 0, 5, 2, 11, 129, 129, 12, 103, 4, 0, 0, 2, 31, 129, 129, 94, 4, 0, 0, 0, 3, 13, 42, 0, 15, 38, 2, 70, 129, 1, 0, 5, 10, 40, 12, 74, 129, 6, 1, 129, 39, 6, 1, 2, 22, 9, 33, 122, 13, 0, 0, 0, 0, 5, 23, 4, 11, 9, 12, 45, 38, 1, 0, 0, 4, 36, 38, 57, 32, 0, 0, 82, 22, 9, 5, 13, 11, 3, 94, 35, 3, 0, 0, 0, 1, 16, 97] 5 +drop table vector_index_01; create table vector_index_02(a bigint primary key, b vecf32(128),c int,key c_k(c)); insert into vector_index_02 values(9774 ,"[1, 0, 1, 6, 6, 17, 47, 39, 2, 0, 1, 25, 27, 10, 56, 130, 18, 5, 2, 6, 15, 2, 19, 130, 42, 28, 1, 1, 2, 1, 0, 5, 0, 2, 4, 4, 31, 34, 44, 35, 9, 3, 8, 11, 33, 12, 61, 130, 130, 17, 0, 1, 6, 2, 9, 130, 111, 36, 0, 0, 11, 9, 1, 12, 2, 100, 130, 28, 7, 2, 6, 7, 9, 27, 130, 83, 5, 0, 1, 18, 130, 130, 84, 9, 0, 0, 2, 24, 111, 24, 0, 1, 37, 24, 2, 10, 12, 62, 33, 3, 0, 0, 0, 1, 3, 16, 106, 28, 0, 0, 0, 0, 17, 46, 85, 10, 0, 0, 1, 4, 11, 4, 2, 2, 9, 14, 8, 8]",3),(9775,"[0, 1, 1, 3, 0, 3, 46, 20, 1, 4, 17, 9, 1, 17, 108, 15, 0, 3, 37, 17, 6, 15, 116, 16, 6, 1, 4, 7, 7, 7, 9, 6, 0, 8, 10, 4, 26, 129, 27, 9, 0, 0, 5, 2, 11, 129, 129, 12, 103, 4, 0, 0, 2, 31, 129, 129, 94, 4, 0, 0, 0, 3, 13, 42, 0, 15, 38, 2, 70, 129, 1, 0, 5, 10, 40, 12, 74, 129, 6, 1, 129, 39, 6, 1, 2, 22, 9, 33, 122, 13, 0, 0, 0, 0, 5, 23, 4, 11, 9, 12, 45, 38, 1, 0, 0, 4, 36, 38, 57, 32, 0, 0, 82, 22, 9, 5, 13, 11, 3, 94, 35, 3, 0, 0, 0, 1, 16, 97]",5),(9776,"[10, 3, 8, 5, 48, 26, 5, 16, 17, 0, 0, 2, 132, 53, 1, 16, 112, 6, 0, 0, 7, 2, 1, 48, 48, 15, 18, 31, 3, 0, 0, 9, 6, 10, 19, 27, 50, 46, 17, 9, 18, 1, 4, 48, 132, 23, 3, 5, 132, 9, 4, 3, 11, 0, 2, 46, 84, 12, 10, 10, 1, 0, 12, 76, 26, 22, 16, 26, 35, 15, 3, 16, 15, 1, 51, 132, 125, 8, 1, 2, 132, 51, 67, 91, 8, 0, 0, 30, 126, 39, 32, 38, 4, 0, 1, 12, 24, 2, 2, 2, 4, 7, 2, 19, 93, 19, 70, 92, 2, 3, 1, 21, 36, 58, 132, 94, 0, 0, 0, 0, 21, 25, 57, 48, 1, 0, 0, 1]",3); insert into vector_index_02 values(9777, "[16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]",4),(9778,"[41, 0, 0, 7, 1, 1, 20, 67, 9, 0, 0, 0, 0, 31, 120, 61, 25, 0, 0, 0, 0, 10, 120, 90, 32, 0, 0, 1, 13, 11, 22, 50, 4, 0, 2, 93, 40, 15, 37, 18, 12, 2, 2, 19, 8, 44, 120, 25, 120, 5, 0, 0, 0, 2, 48, 97, 102, 14, 3, 3, 11, 9, 34, 41, 0, 0, 4, 120, 56, 3, 4, 5, 6, 15, 37, 116, 28, 0, 0, 3, 120, 120, 24, 6, 2, 0, 1, 28, 53, 90, 51, 11, 11, 2, 12, 14, 8, 6, 4, 30, 9, 1, 4, 22, 25, 79, 120, 66, 5, 0, 0, 6, 42, 120, 91, 43, 15, 2, 4, 39, 12, 9, 9, 12, 15, 5, 24, 36]",4); @@ -302,8 +310,8 @@ a b c 9776 [10, 3, 8, 5, 48, 26, 5, 16, 17, 0, 0, 2, 132, 53, 1, 16, 112, 6, 0, 0, 7, 2, 1, 48, 48, 15, 18, 31, 3, 0, 0, 9, 6, 10, 19, 27, 50, 46, 17, 9, 18, 1, 4, 48, 132, 23, 3, 5, 132, 9, 4, 3, 11, 0, 2, 46, 84, 12, 10, 10, 1, 0, 12, 76, 26, 22, 16, 26, 35, 15, 3, 16, 15, 1, 51, 132, 125, 8, 1, 2, 132, 51, 67, 91, 8, 0, 0, 30, 126, 39, 32, 38, 4, 0, 1, 12, 24, 2, 2, 2, 4, 7, 2, 19, 93, 19, 70, 92, 2, 3, 1, 21, 36, 58, 132, 94, 0, 0, 0, 0, 21, 25, 57, 48, 1, 0, 0, 1] 3 select *, cosine_distance(b, "[1, 15, 15, 0, 5, 7, 5, 5, 4, 0, 0, 0, 28, 1, 12, 5, 75, 20, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") as orderbyfn from vector_cos_01 order by orderbyfn ASC LIMIT 2; a b c orderbyfn -9777 [16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13] 4 0.03196156647468429 -9776 [10, 3, 8, 5, 48, 26, 5, 16, 17, 0, 0, 2, 132, 53, 1, 16, 112, 6, 0, 0, 7, 2, 1, 48, 48, 15, 18, 31, 3, 0, 0, 9, 6, 10, 19, 27, 50, 46, 17, 9, 18, 1, 4, 48, 132, 23, 3, 5, 132, 9, 4, 3, 11, 0, 2, 46, 84, 12, 10, 10, 1, 0, 12, 76, 26, 22, 16, 26, 35, 15, 3, 16, 15, 1, 51, 132, 125, 8, 1, 2, 132, 51, 67, 91, 8, 0, 0, 30, 126, 39, 32, 38, 4, 0, 1, 12, 24, 2, 2, 2, 4, 7, 2, 19, 93, 19, 70, 92, 2, 3, 1, 21, 36, 58, 132, 94, 0, 0, 0, 0, 21, 25, 57, 48, 1, 0, 0, 1] 3 0.2662930237053135 +9777 [16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13] 4 0.031961798667907715 +9776 [10, 3, 8, 5, 48, 26, 5, 16, 17, 0, 0, 2, 132, 53, 1, 16, 112, 6, 0, 0, 7, 2, 1, 48, 48, 15, 18, 31, 3, 0, 0, 9, 6, 10, 19, 27, 50, 46, 17, 9, 18, 1, 4, 48, 132, 23, 3, 5, 132, 9, 4, 3, 11, 0, 2, 46, 84, 12, 10, 10, 1, 0, 12, 76, 26, 22, 16, 26, 35, 15, 3, 16, 15, 1, 51, 132, 125, 8, 1, 2, 132, 51, 67, 91, 8, 0, 0, 30, 126, 39, 32, 38, 4, 0, 1, 12, 24, 2, 2, 2, 4, 7, 2, 19, 93, 19, 70, 92, 2, 3, 1, 21, 36, 58, 132, 94, 0, 0, 0, 0, 21, 25, 57, 48, 1, 0, 0, 1] 3 0.26629316806793213 select *, l2_distance(b, "[1, 15, 15, 0, 5, 7, 5, 5, 4, 0, 0, 0, 28, 1, 12, 5, 75, 20, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") as orderbyfn from vector_cos_01 order by cosine_distance(b, "[1, 15, 15, 0, 5, 7, 5, 5, 4, 0, 0, 0, 28, 1, 12, 5, 75, 20, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") ASC LIMIT 2; a b c orderbyfn 9777 [16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13] 4 127.4205551147461 diff --git a/test/distributed/cases/vector/vector_hnsw.sql b/test/distributed/cases/vector/vector_hnsw.sql index fbfd306123153..cccf516e43a1a 100644 --- a/test/distributed/cases/vector/vector_hnsw.sql +++ b/test/distributed/cases/vector/vector_hnsw.sql @@ -18,9 +18,14 @@ insert into vector_index_01 values(9777, " [16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, show create table vector_index_01; desc vector_index_01; --- no result found +select sleep(30); + +-- async index update so result found select * from vector_index_01 order by L2_DISTANCE(b, "[16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") ASC LIMIT 2; select * from vector_index_01 where a>9774 order by L2_DISTANCE(b, "[4, 6, 1, 42, 119, 4, 1, 0, 9, 96, 58, 2, 14, 0, 0, 0, 0, 57, 119, 24, 15, 2, 0, 0, 0, 0, 48, 26, 26, 7, 0, 0, 66, 21, 0, 4, 107, 92, 42, 7, 119, 119, 45, 2, 21, 18, 0, 9, 0, 39, 67, 43, 101, 66, 0, 0, 0, 3, 38, 75, 76, 17, 0, 0, 89, 0, 0, 0, 6, 26, 21, 20, 119, 1, 0, 0, 10, 43, 72, 92, 2, 8, 22, 25, 22, 46, 119, 60, 1, 13, 45, 48, 75, 69, 45, 15, 13, 0, 0, 2, 6, 0, 1, 11, 24, 0, 0, 0, 0, 24, 92, 49, 0, 0, 0, 0, 0, 18, 119, 40, 0, 0, 0, 0, 4, 47, 81, 10]") desc limit 2; + +drop table vector_index_01; + --explain select * from vector_index_01 order by L2_DISTANCE(b, "[1, 15, 15, 0, 5, 7, 5, 5, 4, 0, 0, 0, 28, 1, 12, 5, 75, 20, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") desc LIMIT 2; -- create vector index: create->insert->create index diff --git a/test/distributed/cases/vector/vector_hnsw_async.result b/test/distributed/cases/vector/vector_hnsw_async.result new file mode 100644 index 0000000000000..a849151d687f9 --- /dev/null +++ b/test/distributed/cases/vector/vector_hnsw_async.result @@ -0,0 +1,66 @@ +SET experimental_hnsw_index = 1; +drop database if exists hnsw_cdc; +create database if not exists hnsw_cdc; +use hnsw_cdc; +create table t1(a bigint primary key, b vecf32(3),c int,key c_k(c)); +create index idx01 using hnsw on t1(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; +insert into t1 values (0, "[1,2,3]", 1); +UPDATE t1 set b = '[4,5,6]' where a = 0; +insert into t1 values (1, "[2,3,4]", 1); +DELETE FROM t1 WHERE a=1; +select sleep(30); +sleep(30) +0 +select * from t1 order by L2_DISTANCE(b,"[1,2,3]") ASC LIMIT 10; +a b c +0 [4, 5, 6] 1 +select * from t1 order by L2_DISTANCE(b,"[4,5,6]") ASC LIMIT 10; +a b c +0 [4, 5, 6] 1 +select * from t1 order by L2_DISTANCE(b,"[2,3,4]") ASC LIMIT 10; +a b c +0 [4, 5, 6] 1 +drop table t1; +create table t2(a bigint primary key, b vecf32(128)); +create index idx2 using hnsw on t2(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table t2 fields terminated by ':' parallel 'true'; +select count(*) from t2; +count(*) +10000 +select sleep(30); +sleep(30) +0 +select * from t2 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; +a b +9999 [14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7] +select * from t2 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; +a b +0 [0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1] +drop table t2; +create table t3(a bigint primary key, b vecf32(128)); +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table t3 fields terminated by ':' parallel 'true'; +select count(*) from t3; +count(*) +10000 +create index idx3 using hnsw on t3(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; +load data infile {'filepath'='$resources/vector/sift128_base_10k_2.csv.gz', 'compression'='gzip'} into table t3 fields terminated by ':' parallel 'true'; +select count(*) from t3; +count(*) +20000 +select sleep(45); +sleep(45) +0 +select * from t3 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; +a b +9999 [14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7] +select * from t3 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; +a b +0 [0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1] +select * from t3 order by L2_DISTANCE(b, "[59, 0, 0, 1, 1, 1, 5, 100, 41, 0, 0, 4, 57, 34, 31, 115, 4, 0, 0, 12, 30, 33, 43, 85, 21, 0, 0, 14, 25, 9, 10, 60, 99, 11, 0, 0, 0, 0, 10, 55, 68, 1, 0, 3, 115, 65, 42, 115, 32, 3, 0, 4, 13, 21, 104, 115, 81, 15, 15, 23, 9, 2, 21, 75, 43, 20, 1, 0, 10, 2, 2, 20, 52, 35, 32, 61, 79, 8, 7, 41, 50, 106, 96, 20, 8, 2, 11, 39, 115, 48, 53, 11, 3, 0, 2, 43, 35, 11, 0, 1, 13, 7, 0, 1, 115, 58, 54, 29, 1, 2, 0, 3, 32, 115, 99, 34, 1, 0, 0, 0, 35, 15, 52, 44, 9, 0, 0, 18]") ASC LIMIT 1; +a b +10000 [59, 0, 0, 1, 1, 1, 5, 100, 41, 0, 0, 4, 57, 34, 31, 115, 4, 0, 0, 12, 30, 33, 43, 85, 21, 0, 0, 14, 25, 9, 10, 60, 99, 11, 0, 0, 0, 0, 10, 55, 68, 1, 0, 3, 115, 65, 42, 115, 32, 3, 0, 4, 13, 21, 104, 115, 81, 15, 15, 23, 9, 2, 21, 75, 43, 20, 1, 0, 10, 2, 2, 20, 52, 35, 32, 61, 79, 8, 7, 41, 50, 106, 96, 20, 8, 2, 11, 39, 115, 48, 53, 11, 3, 0, 2, 43, 35, 11, 0, 1, 13, 7, 0, 1, 115, 58, 54, 29, 1, 2, 0, 3, 32, 115, 99, 34, 1, 0, 0, 0, 35, 15, 52, 44, 9, 0, 0, 18] +select * from t3 order by L2_DISTANCE(b, "[0, 0, 0, 0, 0, 101, 82, 4, 2, 0, 0, 0, 3, 133, 133, 8, 46, 1, 2, 13, 15, 29, 87, 50, 22, 1, 0, 16, 25, 6, 18, 49, 5, 2, 0, 2, 3, 59, 70, 19, 18, 2, 0, 11, 42, 37, 30, 13, 133, 13, 4, 53, 28, 3, 8, 42, 77, 6, 11, 103, 36, 0, 0, 32, 7, 15, 59, 27, 2, 0, 2, 5, 14, 5, 55, 52, 51, 3, 2, 5, 133, 21, 10, 38, 26, 1, 0, 64, 71, 3, 10, 118, 53, 5, 6, 28, 33, 26, 73, 15, 0, 0, 0, 22, 13, 15, 133, 133, 4, 0, 0, 15, 107, 62, 46, 91, 9, 1, 7, 16, 28, 4, 0, 27, 33, 4, 15, 25]") ASC LIMIT 1; +a b +19999 [0, 0, 0, 0, 0, 101, 82, 4, 2, 0, 0, 0, 3, 133, 133, 8, 46, 1, 2, 13, 15, 29, 87, 50, 22, 1, 0, 16, 25, 6, 18, 49, 5, 2, 0, 2, 3, 59, 70, 19, 18, 2, 0, 11, 42, 37, 30, 13, 133, 13, 4, 53, 28, 3, 8, 42, 77, 6, 11, 103, 36, 0, 0, 32, 7, 15, 59, 27, 2, 0, 2, 5, 14, 5, 55, 52, 51, 3, 2, 5, 133, 21, 10, 38, 26, 1, 0, 64, 71, 3, 10, 118, 53, 5, 6, 28, 33, 26, 73, 15, 0, 0, 0, 22, 13, 15, 133, 133, 4, 0, 0, 15, 107, 62, 46, 91, 9, 1, 7, 16, 28, 4, 0, 27, 33, 4, 15, 25] +drop table t3; +drop database hnsw_cdc; diff --git a/test/distributed/cases/vector/vector_hnsw_async.sql b/test/distributed/cases/vector/vector_hnsw_async.sql new file mode 100644 index 0000000000000..79f9cef553d95 --- /dev/null +++ b/test/distributed/cases/vector/vector_hnsw_async.sql @@ -0,0 +1,96 @@ + +SET experimental_hnsw_index = 1; + +drop database if exists hnsw_cdc; +create database if not exists hnsw_cdc; +use hnsw_cdc; + +create table t1(a bigint primary key, b vecf32(3),c int,key c_k(c)); + +-- empty data +create index idx01 using hnsw on t1(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; + +-- select sleep(30); + +insert into t1 values (0, "[1,2,3]", 1); +-- select hnsw_cdc_update('hnsw_cdc', 't1', 3, '{"start":"", "end":"", "cdc":[{"t":"U", "pk":0, "v":[1,2,3]}]}'); + +UPDATE t1 set b = '[4,5,6]' where a = 0; +-- select hnsw_cdc_update('hnsw_cdc', 't1', 3, '{"start":"", "end":"", "cdc":[{"t":"U", "pk":0, "v":[4,5,6]}]}'); + +insert into t1 values (1, "[2,3,4]", 1); +-- select hnsw_cdc_update('hnsw_cdc', 't1', 3, '{"start":"", "end":"", "cdc":[{"t":"I", "pk":1, "v":[2,3,4]}]}'); + +DELETE FROM t1 WHERE a=1; +-- select hnsw_cdc_update('hnsw_cdc', 't1', 3, '{"start":"", "end":"", "cdc":[{"t":"D", "pk":0}]}'); + +select sleep(30); + +-- test with multi-cn is tricky. since model is cached in memory, model may not be updated after CDC sync'd. The only way to test is to all INSERT/DELETE/UPDATE before SELECT. +-- already update to [4,5,6], result is [4,5,6] +select * from t1 order by L2_DISTANCE(b,"[1,2,3]") ASC LIMIT 10; + +-- should return a=0 +select * from t1 order by L2_DISTANCE(b,"[4,5,6]") ASC LIMIT 10; + +-- a=1 deleted. result is [4,5,6] +select * from t1 order by L2_DISTANCE(b,"[2,3,4]") ASC LIMIT 10; + +drop table t1; + +-- t2 +create table t2(a bigint primary key, b vecf32(128)); +create index idx2 using hnsw on t2(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; +-- select sleep(30); + +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table t2 fields terminated by ':' parallel 'true'; + +select count(*) from t2; + +select sleep(30); + +select * from t2 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; + +select * from t2 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; + + +-- delete whole table won't work for now. +-- delete from t2 +-- select sleep(10) + +drop table t2; + +-- end t2 + +-- t3 +create table t3(a bigint primary key, b vecf32(128)); + +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table t3 fields terminated by ':' parallel 'true'; + +select count(*) from t3; + +create index idx3 using hnsw on t3(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; + +-- select sleep(30); + +load data infile {'filepath'='$resources/vector/sift128_base_10k_2.csv.gz', 'compression'='gzip'} into table t3 fields terminated by ':' parallel 'true'; + +select count(*) from t3; + +select sleep(45); + +select * from t3 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; + +select * from t3 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; + + +select * from t3 order by L2_DISTANCE(b, "[59, 0, 0, 1, 1, 1, 5, 100, 41, 0, 0, 4, 57, 34, 31, 115, 4, 0, 0, 12, 30, 33, 43, 85, 21, 0, 0, 14, 25, 9, 10, 60, 99, 11, 0, 0, 0, 0, 10, 55, 68, 1, 0, 3, 115, 65, 42, 115, 32, 3, 0, 4, 13, 21, 104, 115, 81, 15, 15, 23, 9, 2, 21, 75, 43, 20, 1, 0, 10, 2, 2, 20, 52, 35, 32, 61, 79, 8, 7, 41, 50, 106, 96, 20, 8, 2, 11, 39, 115, 48, 53, 11, 3, 0, 2, 43, 35, 11, 0, 1, 13, 7, 0, 1, 115, 58, 54, 29, 1, 2, 0, 3, 32, 115, 99, 34, 1, 0, 0, 0, 35, 15, 52, 44, 9, 0, 0, 18]") ASC LIMIT 1; + +select * from t3 order by L2_DISTANCE(b, "[0, 0, 0, 0, 0, 101, 82, 4, 2, 0, 0, 0, 3, 133, 133, 8, 46, 1, 2, 13, 15, 29, 87, 50, 22, 1, 0, 16, 25, 6, 18, 49, 5, 2, 0, 2, 3, 59, 70, 19, 18, 2, 0, 11, 42, 37, 30, 13, 133, 13, 4, 53, 28, 3, 8, 42, 77, 6, 11, 103, 36, 0, 0, 32, 7, 15, 59, 27, 2, 0, 2, 5, 14, 5, 55, 52, 51, 3, 2, 5, 133, 21, 10, 38, 26, 1, 0, 64, 71, 3, 10, 118, 53, 5, 6, 28, 33, 26, 73, 15, 0, 0, 0, 22, 13, 15, 133, 133, 4, 0, 0, 15, 107, 62, 46, 91, 9, 1, 7, 16, 28, 4, 0, 27, 33, 4, 15, 25]") ASC LIMIT 1; + +drop table t3; + +-- end t3 + +drop database hnsw_cdc; + diff --git a/test/distributed/cases/vector/vector_hnsw_f64_async.result b/test/distributed/cases/vector/vector_hnsw_f64_async.result new file mode 100644 index 0000000000000..8f56e173eb72f --- /dev/null +++ b/test/distributed/cases/vector/vector_hnsw_f64_async.result @@ -0,0 +1,66 @@ +SET experimental_hnsw_index = 1; +drop database if exists hnsw_cdc; +create database if not exists hnsw_cdc; +use hnsw_cdc; +create table t1(a bigint primary key, b vecf64(3),c int,key c_k(c)); +create index idx01 using hnsw on t1(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; +insert into t1 values (0, "[1,2,3]", 1); +UPDATE t1 set b = '[4,5,6]' where a = 0; +insert into t1 values (1, "[2,3,4]", 1); +DELETE FROM t1 WHERE a=1; +select sleep(30); +sleep(30) +0 +select * from t1 order by L2_DISTANCE(b,"[1,2,3]") ASC LIMIT 10; +a b c +0 [4, 5, 6] 1 +select * from t1 order by L2_DISTANCE(b,"[4,5,6]") ASC LIMIT 10; +a b c +0 [4, 5, 6] 1 +select * from t1 order by L2_DISTANCE(b,"[2,3,4]") ASC LIMIT 10; +a b c +0 [4, 5, 6] 1 +drop table t1; +create table t2(a bigint primary key, b vecf64(128)); +create index idx2 using hnsw on t2(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table t2 fields terminated by ':' parallel 'true'; +select count(*) from t2; +count(*) +10000 +select sleep(30); +sleep(30) +0 +select * from t2 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; +a b +9999 [14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7] +select * from t2 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; +a b +0 [0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1] +drop table t2; +create table t3(a bigint primary key, b vecf64(128)); +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table t3 fields terminated by ':' parallel 'true'; +select count(*) from t3; +count(*) +10000 +create index idx3 using hnsw on t3(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; +load data infile {'filepath'='$resources/vector/sift128_base_10k_2.csv.gz', 'compression'='gzip'} into table t3 fields terminated by ':' parallel 'true'; +select count(*) from t3; +count(*) +20000 +select sleep(45); +sleep(45) +0 +select * from t3 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; +a b +9999 [14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7] +select * from t3 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; +a b +0 [0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1] +select * from t3 order by L2_DISTANCE(b, "[59, 0, 0, 1, 1, 1, 5, 100, 41, 0, 0, 4, 57, 34, 31, 115, 4, 0, 0, 12, 30, 33, 43, 85, 21, 0, 0, 14, 25, 9, 10, 60, 99, 11, 0, 0, 0, 0, 10, 55, 68, 1, 0, 3, 115, 65, 42, 115, 32, 3, 0, 4, 13, 21, 104, 115, 81, 15, 15, 23, 9, 2, 21, 75, 43, 20, 1, 0, 10, 2, 2, 20, 52, 35, 32, 61, 79, 8, 7, 41, 50, 106, 96, 20, 8, 2, 11, 39, 115, 48, 53, 11, 3, 0, 2, 43, 35, 11, 0, 1, 13, 7, 0, 1, 115, 58, 54, 29, 1, 2, 0, 3, 32, 115, 99, 34, 1, 0, 0, 0, 35, 15, 52, 44, 9, 0, 0, 18]") ASC LIMIT 1; +a b +10000 [59, 0, 0, 1, 1, 1, 5, 100, 41, 0, 0, 4, 57, 34, 31, 115, 4, 0, 0, 12, 30, 33, 43, 85, 21, 0, 0, 14, 25, 9, 10, 60, 99, 11, 0, 0, 0, 0, 10, 55, 68, 1, 0, 3, 115, 65, 42, 115, 32, 3, 0, 4, 13, 21, 104, 115, 81, 15, 15, 23, 9, 2, 21, 75, 43, 20, 1, 0, 10, 2, 2, 20, 52, 35, 32, 61, 79, 8, 7, 41, 50, 106, 96, 20, 8, 2, 11, 39, 115, 48, 53, 11, 3, 0, 2, 43, 35, 11, 0, 1, 13, 7, 0, 1, 115, 58, 54, 29, 1, 2, 0, 3, 32, 115, 99, 34, 1, 0, 0, 0, 35, 15, 52, 44, 9, 0, 0, 18] +select * from t3 order by L2_DISTANCE(b, "[0, 0, 0, 0, 0, 101, 82, 4, 2, 0, 0, 0, 3, 133, 133, 8, 46, 1, 2, 13, 15, 29, 87, 50, 22, 1, 0, 16, 25, 6, 18, 49, 5, 2, 0, 2, 3, 59, 70, 19, 18, 2, 0, 11, 42, 37, 30, 13, 133, 13, 4, 53, 28, 3, 8, 42, 77, 6, 11, 103, 36, 0, 0, 32, 7, 15, 59, 27, 2, 0, 2, 5, 14, 5, 55, 52, 51, 3, 2, 5, 133, 21, 10, 38, 26, 1, 0, 64, 71, 3, 10, 118, 53, 5, 6, 28, 33, 26, 73, 15, 0, 0, 0, 22, 13, 15, 133, 133, 4, 0, 0, 15, 107, 62, 46, 91, 9, 1, 7, 16, 28, 4, 0, 27, 33, 4, 15, 25]") ASC LIMIT 1; +a b +19999 [0, 0, 0, 0, 0, 101, 82, 4, 2, 0, 0, 0, 3, 133, 133, 8, 46, 1, 2, 13, 15, 29, 87, 50, 22, 1, 0, 16, 25, 6, 18, 49, 5, 2, 0, 2, 3, 59, 70, 19, 18, 2, 0, 11, 42, 37, 30, 13, 133, 13, 4, 53, 28, 3, 8, 42, 77, 6, 11, 103, 36, 0, 0, 32, 7, 15, 59, 27, 2, 0, 2, 5, 14, 5, 55, 52, 51, 3, 2, 5, 133, 21, 10, 38, 26, 1, 0, 64, 71, 3, 10, 118, 53, 5, 6, 28, 33, 26, 73, 15, 0, 0, 0, 22, 13, 15, 133, 133, 4, 0, 0, 15, 107, 62, 46, 91, 9, 1, 7, 16, 28, 4, 0, 27, 33, 4, 15, 25] +drop table t3; +drop database hnsw_cdc; diff --git a/test/distributed/cases/vector/vector_hnsw_f64_async.sql b/test/distributed/cases/vector/vector_hnsw_f64_async.sql new file mode 100644 index 0000000000000..6bf3192ff9e3d --- /dev/null +++ b/test/distributed/cases/vector/vector_hnsw_f64_async.sql @@ -0,0 +1,96 @@ + +SET experimental_hnsw_index = 1; + +drop database if exists hnsw_cdc; +create database if not exists hnsw_cdc; +use hnsw_cdc; + +create table t1(a bigint primary key, b vecf64(3),c int,key c_k(c)); + +-- empty data +create index idx01 using hnsw on t1(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; + +-- select sleep(30); + +insert into t1 values (0, "[1,2,3]", 1); +-- select hnsw_cdc_update('hnsw_cdc', 't1', 3, '{"start":"", "end":"", "cdc":[{"t":"U", "pk":0, "v":[1,2,3]}]}'); + +UPDATE t1 set b = '[4,5,6]' where a = 0; +-- select hnsw_cdc_update('hnsw_cdc', 't1', 3, '{"start":"", "end":"", "cdc":[{"t":"U", "pk":0, "v":[4,5,6]}]}'); + +insert into t1 values (1, "[2,3,4]", 1); +-- select hnsw_cdc_update('hnsw_cdc', 't1', 3, '{"start":"", "end":"", "cdc":[{"t":"I", "pk":1, "v":[2,3,4]}]}'); + +DELETE FROM t1 WHERE a=1; +-- select hnsw_cdc_update('hnsw_cdc', 't1', 3, '{"start":"", "end":"", "cdc":[{"t":"D", "pk":0}]}'); + +select sleep(30); + +-- test with multi-cn is tricky. since model is cached in memory, model may not be updated after CDC sync'd. The only way to test is to all INSERT/DELETE/UPDATE before SELECT. +-- already update to [4,5,6], result is [4,5,6] +select * from t1 order by L2_DISTANCE(b,"[1,2,3]") ASC LIMIT 10; + +-- should return a=0 +select * from t1 order by L2_DISTANCE(b,"[4,5,6]") ASC LIMIT 10; + +-- a=1 deleted. result is [4,5,6] +select * from t1 order by L2_DISTANCE(b,"[2,3,4]") ASC LIMIT 10; + +drop table t1; + +-- t2 +create table t2(a bigint primary key, b vecf64(128)); +create index idx2 using hnsw on t2(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; +-- select sleep(30); + +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table t2 fields terminated by ':' parallel 'true'; + +select count(*) from t2; + +select sleep(30); + +select * from t2 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; + +select * from t2 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; + + +-- delete whole table won't work for now. +-- delete from t2 +-- select sleep(10) + +drop table t2; + +-- end t2 + +-- t3 +create table t3(a bigint primary key, b vecf64(128)); + +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table t3 fields terminated by ':' parallel 'true'; + +select count(*) from t3; + +create index idx3 using hnsw on t3(b) op_type "vector_l2_ops" M 48 EF_CONSTRUCTION 64 EF_SEARCH 64 ASYNC; + +-- select sleep(30); + +load data infile {'filepath'='$resources/vector/sift128_base_10k_2.csv.gz', 'compression'='gzip'} into table t3 fields terminated by ':' parallel 'true'; + +select count(*) from t3; + +select sleep(45); + +select * from t3 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; + +select * from t3 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; + + +select * from t3 order by L2_DISTANCE(b, "[59, 0, 0, 1, 1, 1, 5, 100, 41, 0, 0, 4, 57, 34, 31, 115, 4, 0, 0, 12, 30, 33, 43, 85, 21, 0, 0, 14, 25, 9, 10, 60, 99, 11, 0, 0, 0, 0, 10, 55, 68, 1, 0, 3, 115, 65, 42, 115, 32, 3, 0, 4, 13, 21, 104, 115, 81, 15, 15, 23, 9, 2, 21, 75, 43, 20, 1, 0, 10, 2, 2, 20, 52, 35, 32, 61, 79, 8, 7, 41, 50, 106, 96, 20, 8, 2, 11, 39, 115, 48, 53, 11, 3, 0, 2, 43, 35, 11, 0, 1, 13, 7, 0, 1, 115, 58, 54, 29, 1, 2, 0, 3, 32, 115, 99, 34, 1, 0, 0, 0, 35, 15, 52, 44, 9, 0, 0, 18]") ASC LIMIT 1; + +select * from t3 order by L2_DISTANCE(b, "[0, 0, 0, 0, 0, 101, 82, 4, 2, 0, 0, 0, 3, 133, 133, 8, 46, 1, 2, 13, 15, 29, 87, 50, 22, 1, 0, 16, 25, 6, 18, 49, 5, 2, 0, 2, 3, 59, 70, 19, 18, 2, 0, 11, 42, 37, 30, 13, 133, 13, 4, 53, 28, 3, 8, 42, 77, 6, 11, 103, 36, 0, 0, 32, 7, 15, 59, 27, 2, 0, 2, 5, 14, 5, 55, 52, 51, 3, 2, 5, 133, 21, 10, 38, 26, 1, 0, 64, 71, 3, 10, 118, 53, 5, 6, 28, 33, 26, 73, 15, 0, 0, 0, 22, 13, 15, 133, 133, 4, 0, 0, 15, 107, 62, 46, 91, 9, 1, 7, 16, 28, 4, 0, 27, 33, 4, 15, 25]") ASC LIMIT 1; + +drop table t3; + +-- end t3 + +drop database hnsw_cdc; + diff --git a/test/distributed/cases/vector/vector_ivf_async.result b/test/distributed/cases/vector/vector_ivf_async.result new file mode 100644 index 0000000000000..f8f6b63126124 --- /dev/null +++ b/test/distributed/cases/vector/vector_ivf_async.result @@ -0,0 +1,61 @@ +SET experimental_ivf_index = 0; +create table vector_index_00(a int primary key, b vecf32(128),c int,key c_k(c), KEY idx using ivfflat(b) lists = 256 op_type 'vector_l2_ops'); +internal error: experimental_ivf_index is not enabled +SET experimental_ivf_index = 1; +SET probe_limit=1; +create table vector_index_01(a int primary key, b vecf32(128),c int,key c_k(c)); +insert into vector_index_01 values(9774 ,"[1, 0, 1, 6, 6, 17, 47, 39, 2, 0, 1, 25, 27, 10, 56, 130, 18, 5, 2, 6, 15, 2, 19, 130, 42, 28, 1, 1, 2, 1, 0, 5, 0, 2, 4, 4, 31, 34, 44, 35, 9, 3, 8, 11, 33, 12, 61, 130, 130, 17, 0, 1, 6, 2, 9, 130, 111, 36, 0, 0, 11, 9, 1, 12, 2, 100, 130, 28, 7, 2, 6, 7, 9, 27, 130, 83, 5, 0, 1, 18, 130, 130, 84, 9, 0, 0, 2, 24, 111, 24, 0, 1, 37, 24, 2, 10, 12, 62, 33, 3, 0, 0, 0, 1, 3, 16, 106, 28, 0, 0, 0, 0, 17, 46, 85, 10, 0, 0, 1, 4, 11, 4, 2, 2, 9, 14, 8, 8]",3),(9775,"[0, 1, 1, 3, 0, 3, 46, 20, 1, 4, 17, 9, 1, 17, 108, 15, 0, 3, 37, 17, 6, 15, 116, 16, 6, 1, 4, 7, 7, 7, 9, 6, 0, 8, 10, 4, 26, 129, 27, 9, 0, 0, 5, 2, 11, 129, 129, 12, 103, 4, 0, 0, 2, 31, 129, 129, 94, 4, 0, 0, 0, 3, 13, 42, 0, 15, 38, 2, 70, 129, 1, 0, 5, 10, 40, 12, 74, 129, 6, 1, 129, 39, 6, 1, 2, 22, 9, 33, 122, 13, 0, 0, 0, 0, 5, 23, 4, 11, 9, 12, 45, 38, 1, 0, 0, 4, 36, 38, 57, 32, 0, 0, 82, 22, 9, 5, 13, 11, 3, 94, 35, 3, 0, 0, 0, 1, 16, 97]",5),(9776,"[10, 3, 8, 5, 48, 26, 5, 16, 17, 0, 0, 2, 132, 53, 1, 16, 112, 6, 0, 0, 7, 2, 1, 48, 48, 15, 18, 31, 3, 0, 0, 9, 6, 10, 19, 27, 50, 46, 17, 9, 18, 1, 4, 48, 132, 23, 3, 5, 132, 9, 4, 3, 11, 0, 2, 46, 84, 12, 10, 10, 1, 0, 12, 76, 26, 22, 16, 26, 35, 15, 3, 16, 15, 1, 51, 132, 125, 8, 1, 2, 132, 51, 67, 91, 8, 0, 0, 30, 126, 39, 32, 38, 4, 0, 1, 12, 24, 2, 2, 2, 4, 7, 2, 19, 93, 19, 70, 92, 2, 3, 1, 21, 36, 58, 132, 94, 0, 0, 0, 0, 21, 25, 57, 48, 1, 0, 0, 1]",3); +insert into vector_index_01 values(9777, " [16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]",4),(9778,"[41, 0, 0, 7, 1, 1, 20, 67, 9, 0, 0, 0, 0, 31, 120, 61, 25, 0, 0, 0, 0, 10, 120, 90, 32, 0, 0, 1, 13, 11, 22, 50, 4, 0, 2, 93, 40, 15, 37, 18, 12, 2, 2, 19, 8, 44, 120, 25, 120, 5, 0, 0, 0, 2, 48, 97, 102, 14, 3, 3, 11, 9, 34, 41, 0, 0, 4, 120, 56, 3, 4, 5, 6, 15, 37, 116, 28, 0, 0, 3, 120, 120, 24, 6, 2, 0, 1, 28, 53, 90, 51, 11, 11, 2, 12, 14, 8, 6, 4, 30, 9, 1, 4, 22, 25, 79, 120, 66, 5, 0, 0, 6, 42, 120, 91, 43, 15, 2, 4, 39, 12, 9, 9, 12, 15, 5, 24, 36]",4); +create index idx01 using ivfflat on vector_index_01(b) lists=5 op_type "vector_l2_ops" ASYNC; +show create table vector_index_01; +Table Create Table +vector_index_01 CREATE TABLE `vector_index_01` (\n `a` int NOT NULL,\n `b` vecf32(128) DEFAULT NULL,\n `c` int DEFAULT NULL,\n PRIMARY KEY (`a`),\n KEY `c_k` (`c`),\n KEY `idx01` USING ivfflat (`b`) lists = 5 op_type 'vector_l2_ops' async \n) +desc vector_index_01; +Field Type Null Key Default Extra Comment +a INT(32) NO PRI null +b VECF32(128) YES MUL null +c INT(32) YES MUL null +select sleep(30); +sleep(30) +0 +select * from vector_index_01 order by L2_DISTANCE(b, "[16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") ASC LIMIT 2; +a b c +9777 [16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13] 4 +select * from vector_index_01 where a>9774 order by L2_DISTANCE(b, "[4, 6, 1, 42, 119, 4, 1, 0, 9, 96, 58, 2, 14, 0, 0, 0, 0, 57, 119, 24, 15, 2, 0, 0, 0, 0, 48, 26, 26, 7, 0, 0, 66, 21, 0, 4, 107, 92, 42, 7, 119, 119, 45, 2, 21, 18, 0, 9, 0, 39, 67, 43, 101, 66, 0, 0, 0, 3, 38, 75, 76, 17, 0, 0, 89, 0, 0, 0, 6, 26, 21, 20, 119, 1, 0, 0, 10, 43, 72, 92, 2, 8, 22, 25, 22, 46, 119, 60, 1, 13, 45, 48, 75, 69, 45, 15, 13, 0, 0, 2, 6, 0, 1, 11, 24, 0, 0, 0, 0, 24, 92, 49, 0, 0, 0, 0, 0, 18, 119, 40, 0, 0, 0, 0, 4, 47, 81, 10]") desc limit 2; +a b c +9775 [0, 1, 1, 3, 0, 3, 46, 20, 1, 4, 17, 9, 1, 17, 108, 15, 0, 3, 37, 17, 6, 15, 116, 16, 6, 1, 4, 7, 7, 7, 9, 6, 0, 8, 10, 4, 26, 129, 27, 9, 0, 0, 5, 2, 11, 129, 129, 12, 103, 4, 0, 0, 2, 31, 129, 129, 94, 4, 0, 0, 0, 3, 13, 42, 0, 15, 38, 2, 70, 129, 1, 0, 5, 10, 40, 12, 74, 129, 6, 1, 129, 39, 6, 1, 2, 22, 9, 33, 122, 13, 0, 0, 0, 0, 5, 23, 4, 11, 9, 12, 45, 38, 1, 0, 0, 4, 36, 38, 57, 32, 0, 0, 82, 22, 9, 5, 13, 11, 3, 94, 35, 3, 0, 0, 0, 1, 16, 97] 5 +insert into vector_index_01 values(9779, " [10, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]",4); +select sleep(30); +sleep(30) +0 +select * from vector_index_01 order by L2_DISTANCE(b, "[10, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") ASC LIMIT 2; +a b c +9779 [10, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13] 4 +9777 [16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13] 4 +drop table vector_index_01; +create table ivf3(a bigint primary key, b vecf32(128)); +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table ivf3 fields terminated by ':' parallel 'true'; +select count(*) from ivf3; +count(*) +10000 +create index idx3 using ivfflat on ivf3(b) op_type "vector_l2_ops" LISTS=100 ASYNC; +load data infile {'filepath'='$resources/vector/sift128_base_10k_2.csv.gz', 'compression'='gzip'} into table ivf3 fields terminated by ':' parallel 'true'; +select count(*) from ivf3; +count(*) +20000 +select sleep(45); +sleep(45) +0 +select * from ivf3 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; +a b +9999 [14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7] +select * from ivf3 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; +a b +0 [0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1] +select * from ivf3 order by L2_DISTANCE(b, "[59, 0, 0, 1, 1, 1, 5, 100, 41, 0, 0, 4, 57, 34, 31, 115, 4, 0, 0, 12, 30, 33, 43, 85, 21, 0, 0, 14, 25, 9, 10, 60, 99, 11, 0, 0, 0, 0, 10, 55, 68, 1, 0, 3, 115, 65, 42, 115, 32, 3, 0, 4, 13, 21, 104, 115, 81, 15, 15, 23, 9, 2, 21, 75, 43, 20, 1, 0, 10, 2, 2, 20, 52, 35, 32, 61, 79, 8, 7, 41, 50, 106, 96, 20, 8, 2, 11, 39, 115, 48, 53, 11, 3, 0, 2, 43, 35, 11, 0, 1, 13, 7, 0, 1, 115, 58, 54, 29, 1, 2, 0, 3, 32, 115, 99, 34, 1, 0, 0, 0, 35, 15, 52, 44, 9, 0, 0, 18]") ASC LIMIT 1; +a b +10000 [59, 0, 0, 1, 1, 1, 5, 100, 41, 0, 0, 4, 57, 34, 31, 115, 4, 0, 0, 12, 30, 33, 43, 85, 21, 0, 0, 14, 25, 9, 10, 60, 99, 11, 0, 0, 0, 0, 10, 55, 68, 1, 0, 3, 115, 65, 42, 115, 32, 3, 0, 4, 13, 21, 104, 115, 81, 15, 15, 23, 9, 2, 21, 75, 43, 20, 1, 0, 10, 2, 2, 20, 52, 35, 32, 61, 79, 8, 7, 41, 50, 106, 96, 20, 8, 2, 11, 39, 115, 48, 53, 11, 3, 0, 2, 43, 35, 11, 0, 1, 13, 7, 0, 1, 115, 58, 54, 29, 1, 2, 0, 3, 32, 115, 99, 34, 1, 0, 0, 0, 35, 15, 52, 44, 9, 0, 0, 18] +select * from ivf3 order by L2_DISTANCE(b, "[0, 0, 0, 0, 0, 101, 82, 4, 2, 0, 0, 0, 3, 133, 133, 8, 46, 1, 2, 13, 15, 29, 87, 50, 22, 1, 0, 16, 25, 6, 18, 49, 5, 2, 0, 2, 3, 59, 70, 19, 18, 2, 0, 11, 42, 37, 30, 13, 133, 13, 4, 53, 28, 3, 8, 42, 77, 6, 11, 103, 36, 0, 0, 32, 7, 15, 59, 27, 2, 0, 2, 5, 14, 5, 55, 52, 51, 3, 2, 5, 133, 21, 10, 38, 26, 1, 0, 64, 71, 3, 10, 118, 53, 5, 6, 28, 33, 26, 73, 15, 0, 0, 0, 22, 13, 15, 133, 133, 4, 0, 0, 15, 107, 62, 46, 91, 9, 1, 7, 16, 28, 4, 0, 27, 33, 4, 15, 25]") ASC LIMIT 1; +a b +19999 [0, 0, 0, 0, 0, 101, 82, 4, 2, 0, 0, 0, 3, 133, 133, 8, 46, 1, 2, 13, 15, 29, 87, 50, 22, 1, 0, 16, 25, 6, 18, 49, 5, 2, 0, 2, 3, 59, 70, 19, 18, 2, 0, 11, 42, 37, 30, 13, 133, 13, 4, 53, 28, 3, 8, 42, 77, 6, 11, 103, 36, 0, 0, 32, 7, 15, 59, 27, 2, 0, 2, 5, 14, 5, 55, 52, 51, 3, 2, 5, 133, 21, 10, 38, 26, 1, 0, 64, 71, 3, 10, 118, 53, 5, 6, 28, 33, 26, 73, 15, 0, 0, 0, 22, 13, 15, 133, 133, 4, 0, 0, 15, 107, 62, 46, 91, 9, 1, 7, 16, 28, 4, 0, 27, 33, 4, 15, 25] +drop table ivf3; diff --git a/test/distributed/cases/vector/vector_ivf_async.sql b/test/distributed/cases/vector/vector_ivf_async.sql new file mode 100644 index 0000000000000..7651be8c6d82c --- /dev/null +++ b/test/distributed/cases/vector/vector_ivf_async.sql @@ -0,0 +1,60 @@ +-- create table error +SET experimental_ivf_index = 0; +create table vector_index_00(a int primary key, b vecf32(128),c int,key c_k(c), KEY idx using ivfflat(b) lists = 256 op_type 'vector_l2_ops'); + + +SET experimental_ivf_index = 1; +SET probe_limit=1; + +-- create vector index: create->create index->insert +create table vector_index_01(a int primary key, b vecf32(128),c int,key c_k(c)); +insert into vector_index_01 values(9774 ,"[1, 0, 1, 6, 6, 17, 47, 39, 2, 0, 1, 25, 27, 10, 56, 130, 18, 5, 2, 6, 15, 2, 19, 130, 42, 28, 1, 1, 2, 1, 0, 5, 0, 2, 4, 4, 31, 34, 44, 35, 9, 3, 8, 11, 33, 12, 61, 130, 130, 17, 0, 1, 6, 2, 9, 130, 111, 36, 0, 0, 11, 9, 1, 12, 2, 100, 130, 28, 7, 2, 6, 7, 9, 27, 130, 83, 5, 0, 1, 18, 130, 130, 84, 9, 0, 0, 2, 24, 111, 24, 0, 1, 37, 24, 2, 10, 12, 62, 33, 3, 0, 0, 0, 1, 3, 16, 106, 28, 0, 0, 0, 0, 17, 46, 85, 10, 0, 0, 1, 4, 11, 4, 2, 2, 9, 14, 8, 8]",3),(9775,"[0, 1, 1, 3, 0, 3, 46, 20, 1, 4, 17, 9, 1, 17, 108, 15, 0, 3, 37, 17, 6, 15, 116, 16, 6, 1, 4, 7, 7, 7, 9, 6, 0, 8, 10, 4, 26, 129, 27, 9, 0, 0, 5, 2, 11, 129, 129, 12, 103, 4, 0, 0, 2, 31, 129, 129, 94, 4, 0, 0, 0, 3, 13, 42, 0, 15, 38, 2, 70, 129, 1, 0, 5, 10, 40, 12, 74, 129, 6, 1, 129, 39, 6, 1, 2, 22, 9, 33, 122, 13, 0, 0, 0, 0, 5, 23, 4, 11, 9, 12, 45, 38, 1, 0, 0, 4, 36, 38, 57, 32, 0, 0, 82, 22, 9, 5, 13, 11, 3, 94, 35, 3, 0, 0, 0, 1, 16, 97]",5),(9776,"[10, 3, 8, 5, 48, 26, 5, 16, 17, 0, 0, 2, 132, 53, 1, 16, 112, 6, 0, 0, 7, 2, 1, 48, 48, 15, 18, 31, 3, 0, 0, 9, 6, 10, 19, 27, 50, 46, 17, 9, 18, 1, 4, 48, 132, 23, 3, 5, 132, 9, 4, 3, 11, 0, 2, 46, 84, 12, 10, 10, 1, 0, 12, 76, 26, 22, 16, 26, 35, 15, 3, 16, 15, 1, 51, 132, 125, 8, 1, 2, 132, 51, 67, 91, 8, 0, 0, 30, 126, 39, 32, 38, 4, 0, 1, 12, 24, 2, 2, 2, 4, 7, 2, 19, 93, 19, 70, 92, 2, 3, 1, 21, 36, 58, 132, 94, 0, 0, 0, 0, 21, 25, 57, 48, 1, 0, 0, 1]",3); +insert into vector_index_01 values(9777, " [16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]",4),(9778,"[41, 0, 0, 7, 1, 1, 20, 67, 9, 0, 0, 0, 0, 31, 120, 61, 25, 0, 0, 0, 0, 10, 120, 90, 32, 0, 0, 1, 13, 11, 22, 50, 4, 0, 2, 93, 40, 15, 37, 18, 12, 2, 2, 19, 8, 44, 120, 25, 120, 5, 0, 0, 0, 2, 48, 97, 102, 14, 3, 3, 11, 9, 34, 41, 0, 0, 4, 120, 56, 3, 4, 5, 6, 15, 37, 116, 28, 0, 0, 3, 120, 120, 24, 6, 2, 0, 1, 28, 53, 90, 51, 11, 11, 2, 12, 14, 8, 6, 4, 30, 9, 1, 4, 22, 25, 79, 120, 66, 5, 0, 0, 6, 42, 120, 91, 43, 15, 2, 4, 39, 12, 9, 9, 12, 15, 5, 24, 36]",4); +create index idx01 using ivfflat on vector_index_01(b) lists=5 op_type "vector_l2_ops" ASYNC; +show create table vector_index_01; +desc vector_index_01; +select sleep(30); +select * from vector_index_01 order by L2_DISTANCE(b, "[16, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") ASC LIMIT 2; +select * from vector_index_01 where a>9774 order by L2_DISTANCE(b, "[4, 6, 1, 42, 119, 4, 1, 0, 9, 96, 58, 2, 14, 0, 0, 0, 0, 57, 119, 24, 15, 2, 0, 0, 0, 0, 48, 26, 26, 7, 0, 0, 66, 21, 0, 4, 107, 92, 42, 7, 119, 119, 45, 2, 21, 18, 0, 9, 0, 39, 67, 43, 101, 66, 0, 0, 0, 3, 38, 75, 76, 17, 0, 0, 89, 0, 0, 0, 6, 26, 21, 20, 119, 1, 0, 0, 10, 43, 72, 92, 2, 8, 22, 25, 22, 46, 119, 60, 1, 13, 45, 48, 75, 69, 45, 15, 13, 0, 0, 2, 6, 0, 1, 11, 24, 0, 0, 0, 0, 24, 92, 49, 0, 0, 0, 0, 0, 18, 119, 40, 0, 0, 0, 0, 4, 47, 81, 10]") desc limit 2; + + +insert into vector_index_01 values(9779, " [10, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]",4); + +select sleep(30); + +-- 9779 +select * from vector_index_01 order by L2_DISTANCE(b, "[10, 15, 0, 0, 5, 46, 5, 5, 4, 0, 0, 0, 28, 118, 12, 5, 75, 44, 5, 0, 6, 32, 6, 49, 41, 74, 9, 1, 0, 0, 0, 9, 1, 9, 16, 41, 71, 80, 3, 0, 0, 4, 3, 5, 51, 106, 11, 3, 112, 28, 13, 1, 4, 8, 3, 104, 118, 14, 1, 1, 0, 0, 0, 88, 3, 27, 46, 118, 108, 49, 2, 0, 1, 46, 118, 118, 27, 12, 0, 0, 33, 118, 118, 8, 0, 0, 0, 4, 118, 95, 40, 0, 0, 0, 1, 11, 27, 38, 12, 12, 18, 29, 3, 2, 13, 30, 94, 78, 30, 19, 9, 3, 31, 45, 70, 42, 15, 1, 3, 12, 14, 22, 16, 2, 3, 17, 24, 13]") ASC LIMIT 2; + +drop table vector_index_01; + + +-- ivf3 + +create table ivf3(a bigint primary key, b vecf32(128)); + +load data infile {'filepath'='$resources/vector/sift128_base_10k.csv.gz', 'compression'='gzip'} into table ivf3 fields terminated by ':' parallel 'true'; + +select count(*) from ivf3; + +create index idx3 using ivfflat on ivf3(b) op_type "vector_l2_ops" LISTS=100 ASYNC; + +-- select sleep(30); + +load data infile {'filepath'='$resources/vector/sift128_base_10k_2.csv.gz', 'compression'='gzip'} into table ivf3 fields terminated by ':' parallel 'true'; + +select count(*) from ivf3; + +select sleep(45); + +select * from ivf3 order by L2_DISTANCE(b, "[14, 2, 0, 0, 0, 2, 42, 55, 9, 1, 0, 0, 18, 100, 77, 32, 89, 1, 0, 0, 19, 85, 15, 68, 52, 4, 0, 0, 0, 0, 2, 28, 34, 13, 5, 12, 49, 40, 39, 37, 24, 2, 0, 0, 34, 83, 88, 28, 119, 20, 0, 0, 41, 39, 13, 62, 119, 16, 2, 0, 0, 0, 10, 42, 9, 46, 82, 79, 64, 19, 2, 5, 10, 35, 26, 53, 84, 32, 34, 9, 119, 119, 21, 3, 3, 11, 17, 14, 119, 25, 8, 5, 0, 0, 11, 22, 23, 17, 42, 49, 17, 12, 5, 5, 12, 78, 119, 90, 27, 0, 4, 2, 48, 92, 112, 85, 15, 0, 2, 7, 50, 36, 15, 11, 1, 0, 0, 7]") ASC LIMIT 1; + +select * from ivf3 order by L2_DISTANCE(b, "[0, 16, 35, 5, 32, 31, 14, 10, 11, 78, 55, 10, 45, 83, 11, 6, 14, 57, 102, 75, 20, 8, 3, 5, 67, 17, 19, 26, 5, 0, 1, 22, 60, 26, 7, 1, 18, 22, 84, 53, 85, 119, 119, 4, 24, 18, 7, 7, 1, 81, 106, 102, 72, 30, 6, 0, 9, 1, 9, 119, 72, 1, 4, 33, 119, 29, 6, 1, 0, 1, 14, 52, 119, 30, 3, 0, 0, 55, 92, 111, 2, 5, 4, 9, 22, 89, 96, 14, 1, 0, 1, 82, 59, 16, 20, 5, 25, 14, 11, 4, 0, 0, 1, 26, 47, 23, 4, 0, 0, 4, 38, 83, 30, 14, 9, 4, 9, 17, 23, 41, 0, 0, 2, 8, 19, 25, 23, 1]") ASC LIMIT 1; + + +select * from ivf3 order by L2_DISTANCE(b, "[59, 0, 0, 1, 1, 1, 5, 100, 41, 0, 0, 4, 57, 34, 31, 115, 4, 0, 0, 12, 30, 33, 43, 85, 21, 0, 0, 14, 25, 9, 10, 60, 99, 11, 0, 0, 0, 0, 10, 55, 68, 1, 0, 3, 115, 65, 42, 115, 32, 3, 0, 4, 13, 21, 104, 115, 81, 15, 15, 23, 9, 2, 21, 75, 43, 20, 1, 0, 10, 2, 2, 20, 52, 35, 32, 61, 79, 8, 7, 41, 50, 106, 96, 20, 8, 2, 11, 39, 115, 48, 53, 11, 3, 0, 2, 43, 35, 11, 0, 1, 13, 7, 0, 1, 115, 58, 54, 29, 1, 2, 0, 3, 32, 115, 99, 34, 1, 0, 0, 0, 35, 15, 52, 44, 9, 0, 0, 18]") ASC LIMIT 1; + +select * from ivf3 order by L2_DISTANCE(b, "[0, 0, 0, 0, 0, 101, 82, 4, 2, 0, 0, 0, 3, 133, 133, 8, 46, 1, 2, 13, 15, 29, 87, 50, 22, 1, 0, 16, 25, 6, 18, 49, 5, 2, 0, 2, 3, 59, 70, 19, 18, 2, 0, 11, 42, 37, 30, 13, 133, 13, 4, 53, 28, 3, 8, 42, 77, 6, 11, 103, 36, 0, 0, 32, 7, 15, 59, 27, 2, 0, 2, 5, 14, 5, 55, 52, 51, 3, 2, 5, 133, 21, 10, 38, 26, 1, 0, 64, 71, 3, 10, 118, 53, 5, 6, 28, 33, 26, 73, 15, 0, 0, 0, 22, 13, 15, 133, 133, 4, 0, 0, 15, 107, 62, 46, 91, 9, 1, 7, 16, 28, 4, 0, 27, 33, 4, 15, 25]") ASC LIMIT 1; + +drop table ivf3; + +