github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/txn/txnimpl/txn_test.go (about)

     1  // Copyright 2021 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package txnimpl
    16  
    17  import (
    18  	"context"
    19  	"fmt"
    20  	"path"
    21  	"sync"
    22  	"testing"
    23  	"time"
    24  
    25  	"github.com/matrixorigin/matrixone/pkg/objectio"
    26  
    27  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    28  	"github.com/matrixorigin/matrixone/pkg/common/mpool"
    29  	"github.com/matrixorigin/matrixone/pkg/container/types"
    30  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog"
    31  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common"
    32  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/dbutils"
    33  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif"
    34  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tables"
    35  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/testutils"
    36  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnbase"
    37  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/wal"
    38  	"github.com/panjf2000/ants/v2"
    39  	"github.com/stretchr/testify/assert"
    40  )
    41  
    42  const (
    43  	ModuleName = "TAETXN"
    44  )
    45  
    46  // 1. 30 concurrency
    47  // 2. 10000 node
    48  // 3. 512K buffer
    49  // 4. 1K(30%), 4K(25%), 8K(%20), 16K(%15), 32K(%10)
    50  
    51  //func getNodes() int {
    52  //	v := rand.Intn(100)
    53  //	if v < 30 {
    54  //		return 1 * 2
    55  //	} else if v < 55 {
    56  //		return 2 * 2
    57  //	} else if v < 75 {
    58  //		return 3 * 2
    59  //	} else if v < 90 {
    60  //		return 4 * 2
    61  //	}
    62  //	return 5 * 2
    63  //}
    64  
    65  //func makeTable(t *testing.T, dir string, colCnt int, pkIdx int, bufSize uint64) *txnTable {
    66  //	mgr := buffer.NewNodeManager(bufSize, nil)
    67  //	driver := wal.NewDriverWithBatchStore(dir, "store", nil)
    68  //	id := common.NextGlobalSeqNum()
    69  //	schema := catalog.MockSchemaAll(colCnt, pkIdx)
    70  //	rel := mockTestRelation(id, schema)
    71  //	txn := txnbase.NewTxn(nil, nil, common.NewTxnIDAllocator().Alloc(), types.NextGlobalTsForTest(), nil)
    72  //	store := newStore(nil, driver, nil, mgr, nil)
    73  //	store.BindTxn(txn)
    74  //	return newTxnTable(store, rel.GetMeta().(*catalog.TableEntry))
    75  //}
    76  
    77  //func TestInsertNode(t *testing.T) {
    78  //	defer testutils.AfterTest(t)()
    79  //	testutils.EnsureNoLeak(t)
    80  //	dir := testutils.InitTestEnv(ModuleName, t)
    81  //	tbl := makeTable(t, dir, 2, 1, mpool.KB*6)
    82  //	defer tbl.store.driver.Close()
    83  //	bat := catalog.MockBatch(tbl.GetSchema(), int(mpool.KB))
    84  //	defer bat.Close()
    85  //	p, _ := ants.NewPool(5)
    86  //	defer p.Release()
    87  //
    88  //	var wg sync.WaitGroup
    89  //	var all uint64
    90  //
    91  //	worker := func(id uint64) func() {
    92  //		return func() {
    93  //			defer wg.Done()
    94  //			cnt := getNodes()
    95  //			nodes := make([]*anode, cnt)
    96  //			for i := 0; i < cnt; i++ {
    97  //				var cid common.ID
    98  //				cid.BlockID = id
    99  //				cid.Idx = uint16(i)
   100  //				n := NewANodeWithID(tbl, tbl.store.nodesMgr, &cid, tbl.store.driver)
   101  //				nodes[i] = n
   102  //				h := tbl.store.nodesMgr.Pin(n.storage.mnode)
   103  //				var err error
   104  //				if err = n.storage.mnode.Expand(mpool.KB*1, func() error {
   105  //					_, err := n.Append(context.Background(), bat, 0)
   106  //					return err
   107  //				}); err != nil {
   108  //					err = n.storage.mnode.Expand(mpool.KB*1, func() error {
   109  //						_, err := n.Append(context.Background(), bat, 0)
   110  //						return err
   111  //					})
   112  //				}
   113  //				if err != nil {
   114  //					assert.NotNil(t, err)
   115  //				}
   116  //				h.Close()
   117  //			}
   118  //			for _, n := range nodes {
   119  //				// n.ToTransient()
   120  //				n.Close()
   121  //			}
   122  //			atomic.AddUint64(&all, uint64(len(nodes)))
   123  //		}
   124  //	}
   125  //	idAlloc := common.NewIdAlloctor(1)
   126  //	for {
   127  //		id := idAlloc.Alloc()
   128  //		if id > 10 {
   129  //			break
   130  //		}
   131  //		wg.Add(1)
   132  //		err := p.Submit(worker(id))
   133  //		assert.Nil(t, err)
   134  //	}
   135  //	wg.Wait()
   136  //	t.Log(all)
   137  //	t.Log(tbl.store.nodesMgr.String())
   138  //}
   139  
   140  func TestTable(t *testing.T) {
   141  	defer testutils.AfterTest(t)()
   142  	ctx := context.Background()
   143  	testutils.EnsureNoLeak(t)
   144  	dir := testutils.InitTestEnv(ModuleName, t)
   145  	c, mgr, driver := initTestContext(ctx, t, dir)
   146  	defer driver.Close()
   147  	defer c.Close()
   148  	defer mgr.Stop()
   149  
   150  	schema := catalog.MockSchemaAll(3, 2)
   151  	schema.BlockMaxRows = 10000
   152  	schema.ObjectMaxBlocks = 10
   153  	{
   154  		txn, _ := mgr.StartTxn(nil)
   155  		db, err := txn.CreateDatabase("db", "", "")
   156  		assert.Nil(t, err)
   157  		rel, _ := db.CreateRelation(schema)
   158  		bat := catalog.MockBatch(schema, int(mpool.KB)*100)
   159  		defer bat.Close()
   160  		bats := bat.Split(100)
   161  		for _, data := range bats {
   162  			err := rel.Append(context.Background(), data)
   163  			assert.Nil(t, err)
   164  		}
   165  		tDB, _ := txn.GetStore().(*txnStore).getOrSetDB(db.GetID())
   166  		tbl, _ := tDB.getOrSetTable(rel.ID())
   167  		err = tbl.RangeDeleteLocalRows(1024+20, 1024+30)
   168  		assert.Nil(t, err)
   169  		err = tbl.RangeDeleteLocalRows(1024*2+38, 1024*2+40)
   170  		assert.Nil(t, err)
   171  		err = tbl.RangeDeleteLocalRows(1024*10+38, 1024*40+40)
   172  		assert.Nil(t, err)
   173  		assert.True(t, tbl.IsLocalDeleted(1024+20))
   174  		assert.True(t, tbl.IsLocalDeleted(1024+30))
   175  		assert.True(t, tbl.IsLocalDeleted(1024*10+38))
   176  		assert.True(t, tbl.IsLocalDeleted(1024*40+40))
   177  		assert.True(t, tbl.IsLocalDeleted(1024*30+40))
   178  		assert.False(t, tbl.IsLocalDeleted(1024+19))
   179  		assert.False(t, tbl.IsLocalDeleted(1024+31))
   180  		assert.False(t, tbl.IsLocalDeleted(1024*10+37))
   181  		assert.False(t, tbl.IsLocalDeleted(1024*40+41))
   182  		err = txn.Commit(context.Background())
   183  		assert.Nil(t, err)
   184  	}
   185  }
   186  
   187  func TestAppend(t *testing.T) {
   188  	defer testutils.AfterTest(t)()
   189  	ctx := context.Background()
   190  	testutils.EnsureNoLeak(t)
   191  	dir := testutils.InitTestEnv(ModuleName, t)
   192  	c, mgr, driver := initTestContext(ctx, t, dir)
   193  	defer driver.Close()
   194  	defer c.Close()
   195  	defer mgr.Stop()
   196  
   197  	schema := catalog.MockSchemaAll(3, 1)
   198  	schema.BlockMaxRows = 10000
   199  	schema.ObjectMaxBlocks = 10
   200  
   201  	txn, _ := mgr.StartTxn(nil)
   202  	db, _ := txn.CreateDatabase("db", "", "")
   203  	rel, _ := db.CreateRelation(schema)
   204  	tDB, _ := txn.GetStore().(*txnStore).getOrSetDB(db.GetID())
   205  	tbl, _ := tDB.getOrSetTable(rel.ID())
   206  	rows := uint64(MaxNodeRows) / 8 * 3
   207  	brows := rows / 3
   208  
   209  	bat := catalog.MockBatch(tbl.GetLocalSchema(), int(rows))
   210  	defer bat.Close()
   211  	bats := bat.Split(3)
   212  
   213  	err := tbl.BatchDedupLocal(bats[0])
   214  	assert.Nil(t, err)
   215  	err = tbl.Append(context.Background(), bats[0])
   216  	assert.Nil(t, err)
   217  	assert.Equal(t, int(brows), int(tbl.UncommittedRows()))
   218  	assert.Equal(t, int(brows), int(tbl.tableSpace.index.Count()))
   219  
   220  	err = tbl.BatchDedupLocal(bats[0])
   221  	assert.NotNil(t, err)
   222  
   223  	err = tbl.BatchDedupLocal(bats[1])
   224  	assert.Nil(t, err)
   225  	err = tbl.Append(context.Background(), bats[1])
   226  	assert.Nil(t, err)
   227  	assert.Equal(t, 2*int(brows), int(tbl.UncommittedRows()))
   228  	assert.Equal(t, 2*int(brows), int(tbl.tableSpace.index.Count()))
   229  
   230  	err = tbl.BatchDedupLocal(bats[2])
   231  	assert.Nil(t, err)
   232  	err = tbl.Append(context.Background(), bats[2])
   233  	assert.Nil(t, err)
   234  	assert.Equal(t, 3*int(brows), int(tbl.UncommittedRows()))
   235  	assert.Equal(t, 3*int(brows), int(tbl.tableSpace.index.Count()))
   236  	assert.NoError(t, txn.Commit(context.Background()))
   237  }
   238  
   239  func TestIndex(t *testing.T) {
   240  	defer testutils.AfterTest(t)()
   241  	testutils.EnsureNoLeak(t)
   242  	index := NewSimpleTableIndex()
   243  	err := index.Insert(1, 10)
   244  	assert.Nil(t, err)
   245  	err = index.Insert("one", 10)
   246  	assert.Nil(t, err)
   247  	row, err := index.Search("one")
   248  	assert.Nil(t, err)
   249  	assert.Equal(t, 10, int(row))
   250  	err = index.Delete("one")
   251  	assert.Nil(t, err)
   252  	_, err = index.Search("one")
   253  	assert.NotNil(t, err)
   254  
   255  	schema := catalog.MockSchemaAll(14, 1)
   256  	bat := catalog.MockBatch(schema, 500)
   257  	defer bat.Close()
   258  
   259  	idx := NewSimpleTableIndex()
   260  	err = idx.BatchDedup(bat.Attrs[0], bat.Vecs[0])
   261  	assert.Nil(t, err)
   262  	err = idx.BatchInsert(bat.Attrs[0], bat.Vecs[0], 0, bat.Vecs[0].Length(), 0, false)
   263  	assert.NotNil(t, err)
   264  
   265  	err = idx.BatchDedup(bat.Attrs[1], bat.Vecs[1])
   266  	assert.Nil(t, err)
   267  	err = idx.BatchInsert(bat.Attrs[1], bat.Vecs[1], 0, bat.Vecs[1].Length(), 0, false)
   268  	assert.Nil(t, err)
   269  
   270  	window := bat.Vecs[1].Window(20, 2)
   271  	assert.Equal(t, 2, window.Length())
   272  	err = idx.BatchDedup(bat.Attrs[1], window)
   273  	assert.NotNil(t, err)
   274  
   275  	schema = catalog.MockSchemaAll(14, 12)
   276  	bat = catalog.MockBatch(schema, 500)
   277  	defer bat.Close()
   278  	idx = NewSimpleTableIndex()
   279  	err = idx.BatchDedup(bat.Attrs[12], bat.Vecs[12])
   280  	assert.Nil(t, err)
   281  	err = idx.BatchInsert(bat.Attrs[12], bat.Vecs[12], 0, bat.Vecs[12].Length(), 0, false)
   282  	assert.Nil(t, err)
   283  
   284  	window = bat.Vecs[12].Window(20, 2)
   285  	assert.Equal(t, 2, window.Length())
   286  	err = idx.BatchDedup(bat.Attrs[12], window)
   287  	assert.Error(t, err)
   288  
   289  	// T_array
   290  	schema = catalog.MockSchemaAll(20, 12)
   291  	bat = catalog.MockBatch(schema, 500)
   292  	defer bat.Close()
   293  	idx = NewSimpleTableIndex()
   294  	err = idx.BatchDedup(bat.Attrs[19], bat.Vecs[19])
   295  	assert.Nil(t, err)
   296  	err = idx.BatchInsert(bat.Attrs[19], bat.Vecs[19], 0, bat.Vecs[19].Length(), 0, false)
   297  	assert.Nil(t, err)
   298  
   299  	window = bat.Vecs[19].Window(20, 2)
   300  	assert.Equal(t, 2, window.Length())
   301  	err = idx.BatchDedup(bat.Attrs[19], window)
   302  	assert.Error(t, err)
   303  }
   304  
   305  func TestLoad(t *testing.T) {
   306  	defer testutils.AfterTest(t)()
   307  	ctx := context.Background()
   308  	testutils.EnsureNoLeak(t)
   309  	dir := testutils.InitTestEnv(ModuleName, t)
   310  	c, mgr, driver := initTestContext(ctx, t, dir)
   311  	defer driver.Close()
   312  	defer c.Close()
   313  	defer mgr.Stop()
   314  
   315  	schema := catalog.MockSchemaAll(14, 13)
   316  	schema.BlockMaxRows = 10000
   317  	schema.ObjectMaxBlocks = 10
   318  
   319  	bat := catalog.MockBatch(schema, 60000)
   320  	defer bat.Close()
   321  	bats := bat.Split(5)
   322  
   323  	txn, _ := mgr.StartTxn(nil)
   324  	db, _ := txn.CreateDatabase("db", "", "")
   325  	rel, _ := db.CreateRelation(schema)
   326  	tDB, _ := txn.GetStore().(*txnStore).getOrSetDB(db.GetID())
   327  	tbl, _ := tDB.getOrSetTable(rel.ID())
   328  
   329  	err := tbl.Append(context.Background(), bats[0])
   330  	assert.NoError(t, err)
   331  
   332  	v, _, err := tbl.GetLocalValue(100, 0)
   333  	assert.NoError(t, err)
   334  	t.Logf("Row %d, Col %d, Val %v", 100, 0, v)
   335  	assert.NoError(t, txn.Commit(context.Background()))
   336  }
   337  
   338  func TestNodeCommand(t *testing.T) {
   339  	defer testutils.AfterTest(t)()
   340  	ctx := context.Background()
   341  	testutils.EnsureNoLeak(t)
   342  	dir := testutils.InitTestEnv(ModuleName, t)
   343  	c, mgr, driver := initTestContext(ctx, t, dir)
   344  	defer driver.Close()
   345  	defer c.Close()
   346  	defer mgr.Stop()
   347  
   348  	schema := catalog.MockSchemaAll(14, 13)
   349  	schema.BlockMaxRows = 10000
   350  	schema.ObjectMaxBlocks = 10
   351  
   352  	bat := catalog.MockBatch(schema, 15000)
   353  	defer bat.Close()
   354  
   355  	txn, _ := mgr.StartTxn(nil)
   356  	db, _ := txn.CreateDatabase("db", "", "")
   357  	rel, _ := db.CreateRelation(schema)
   358  
   359  	tDB, _ := txn.GetStore().(*txnStore).getOrSetDB(db.GetID())
   360  	tbl, _ := tDB.getOrSetTable(rel.ID())
   361  	err := tbl.Append(context.Background(), bat)
   362  	assert.Nil(t, err)
   363  
   364  	err = tbl.RangeDeleteLocalRows(100, 200)
   365  	assert.NoError(t, err)
   366  
   367  	for i, inode := range tbl.tableSpace.nodes {
   368  		cmd, err := inode.MakeCommand(uint32(i))
   369  		assert.NoError(t, err)
   370  		assert.NotNil(t, cmd.(*AppendCmd).Data)
   371  		//if entry != nil {
   372  		//	_ = entry.WaitDone()
   373  		//	entry.Free()
   374  		//}
   375  		if cmd != nil {
   376  			t.Log(cmd.String())
   377  		}
   378  	}
   379  	assert.NoError(t, txn.Commit(context.Background()))
   380  }
   381  
   382  func TestTxnManager1(t *testing.T) {
   383  	defer testutils.AfterTest(t)()
   384  	testutils.EnsureNoLeak(t)
   385  	mgr := txnbase.NewTxnManager(TxnStoreFactory(context.Background(), nil, nil, nil, nil, 0),
   386  		TxnFactory(nil), types.NewMockHLCClock(1))
   387  	mgr.Start(context.Background())
   388  	txn, _ := mgr.StartTxn(nil)
   389  	txn.MockIncWriteCnt()
   390  
   391  	lock := sync.Mutex{}
   392  	seqs := make([]int, 0)
   393  
   394  	txn.SetPrepareCommitFn(func(_ txnif.AsyncTxn) error {
   395  		time.Sleep(time.Millisecond * 100)
   396  		lock.Lock()
   397  		seqs = append(seqs, 2)
   398  		lock.Unlock()
   399  		return nil
   400  	})
   401  
   402  	var wg sync.WaitGroup
   403  	short := func() {
   404  		defer wg.Done()
   405  		txn2, _ := mgr.StartTxn(nil)
   406  		txn2.MockIncWriteCnt()
   407  		txn2.SetPrepareCommitFn(func(_ txnif.AsyncTxn) error {
   408  			lock.Lock()
   409  			seqs = append(seqs, 4)
   410  			lock.Unlock()
   411  			return nil
   412  		})
   413  		time.Sleep(10 * time.Millisecond)
   414  		lock.Lock()
   415  		seqs = append(seqs, 1)
   416  		lock.Unlock()
   417  		txn.GetTxnState(true)
   418  		lock.Lock()
   419  		seqs = append(seqs, 3)
   420  		lock.Unlock()
   421  		err := txn2.Commit(context.Background())
   422  		assert.Nil(t, err)
   423  	}
   424  
   425  	for i := 0; i < 1; i++ {
   426  		wg.Add(1)
   427  		go short()
   428  	}
   429  
   430  	err := txn.Commit(context.Background())
   431  	assert.Nil(t, err)
   432  	wg.Wait()
   433  	defer mgr.Stop()
   434  	expected := []int{1, 2, 3, 4}
   435  	assert.Equal(t, expected, seqs)
   436  }
   437  
   438  func initTestContext(ctx context.Context, t *testing.T, dir string) (*catalog.Catalog, *txnbase.TxnManager, wal.Driver) {
   439  	c := catalog.MockCatalog()
   440  	driver := wal.NewDriverWithBatchStore(context.Background(), dir, "store", nil)
   441  	serviceDir := path.Join(dir, "data")
   442  	service := objectio.TmpNewFileservice(ctx, path.Join(dir, "data"))
   443  	fs := objectio.NewObjectFS(service, serviceDir)
   444  	rt := dbutils.NewRuntime(
   445  		dbutils.WithRuntimeObjectFS(fs),
   446  	)
   447  	factory := tables.NewDataFactory(rt, dir)
   448  	mgr := txnbase.NewTxnManager(TxnStoreFactory(context.Background(), c, driver, rt, factory, 0),
   449  		TxnFactory(c), types.NewMockHLCClock(1))
   450  	rt.Now = mgr.Now
   451  	mgr.Start(context.Background())
   452  	return c, mgr, driver
   453  }
   454  
   455  // 1. Txn1 create database "db" and table "tb1". Commit
   456  // 2. Txn2 drop database
   457  // 3. Txn3 create table "tb2"
   458  // 4. Txn2 commit
   459  // 5. Txn3 commit
   460  func TestTransaction1(t *testing.T) {
   461  	defer testutils.AfterTest(t)()
   462  	ctx := context.Background()
   463  	testutils.EnsureNoLeak(t)
   464  	dir := testutils.InitTestEnv(ModuleName, t)
   465  	c, mgr, driver := initTestContext(ctx, t, dir)
   466  	defer driver.Close()
   467  	defer c.Close()
   468  	defer mgr.Stop()
   469  
   470  	txn1, _ := mgr.StartTxn(nil)
   471  	name := "db"
   472  	schema := catalog.MockSchema(1, 0)
   473  	db, err := txn1.CreateDatabase(name, "", "")
   474  	assert.Nil(t, err)
   475  	_, err = db.CreateRelation(schema)
   476  	assert.Nil(t, err)
   477  	err = txn1.Commit(context.Background())
   478  	assert.Nil(t, err)
   479  	t.Log(c.SimplePPString(common.PPL1))
   480  
   481  	txn2, _ := mgr.StartTxn(nil)
   482  	db2, err := txn2.DropDatabase(name)
   483  	assert.Nil(t, err)
   484  	t.Log(db2.String())
   485  
   486  	txn3, _ := mgr.StartTxn(nil)
   487  	db3, err := txn3.GetDatabase(name)
   488  	assert.Nil(t, err)
   489  	t.Log(db3.String())
   490  	schema = catalog.MockSchema(1, 0)
   491  	rel, err := db3.CreateRelation(schema)
   492  	assert.Nil(t, err)
   493  	t.Log(rel.String())
   494  
   495  	err = txn2.Commit(context.Background())
   496  	assert.Nil(t, err)
   497  	err = txn3.Commit(context.Background())
   498  	assert.NoError(t, err)
   499  	// assert.Equal(t, txnif.TxnStateRollbacked, txn3.GetTxnState(true))
   500  	t.Log(txn3.String())
   501  	t.Log(db2.String())
   502  	t.Log(rel.String())
   503  	t.Log(c.SimplePPString(common.PPL1))
   504  }
   505  
   506  func TestTransaction2(t *testing.T) {
   507  	defer testutils.AfterTest(t)()
   508  	ctx := context.Background()
   509  	testutils.EnsureNoLeak(t)
   510  	dir := testutils.InitTestEnv(ModuleName, t)
   511  	c, mgr, driver := initTestContext(ctx, t, dir)
   512  	defer driver.Close()
   513  	defer c.Close()
   514  	defer mgr.Stop()
   515  
   516  	name := "db"
   517  	txn1, _ := mgr.StartTxn(nil)
   518  	db, err := txn1.CreateDatabase(name, "", "")
   519  	assert.Nil(t, err)
   520  	t.Log(db.String())
   521  
   522  	schema := catalog.MockSchema(1, 0)
   523  	rel, err := db.CreateRelation(schema)
   524  	assert.Nil(t, err)
   525  	t.Log(rel.String())
   526  
   527  	err = txn1.Commit(context.Background())
   528  	assert.Nil(t, err)
   529  	t.Log(db.String())
   530  	assert.Equal(t, txn1.GetCommitTS(), db.GetMeta().(*catalog.DBEntry).GetCreatedAtLocked())
   531  	assert.True(t, db.GetMeta().(*catalog.DBEntry).IsCommittedLocked())
   532  	assert.Equal(t, txn1.GetCommitTS(), rel.GetMeta().(*catalog.TableEntry).GetCreatedAtLocked())
   533  	assert.True(t, rel.GetMeta().(*catalog.TableEntry).IsCommittedLocked())
   534  
   535  	txn2, _ := mgr.StartTxn(nil)
   536  	get, err := txn2.GetDatabase(name)
   537  	assert.Nil(t, err)
   538  	t.Log(get.String())
   539  
   540  	dropped, err := txn2.DropDatabase(name)
   541  	assert.Nil(t, err)
   542  	t.Log(dropped.String())
   543  
   544  	_, err = txn2.GetDatabase(name)
   545  	assert.True(t, moerr.IsMoErrCode(err, moerr.ErrBadDB))
   546  	t.Log(err)
   547  
   548  	txn3, _ := mgr.StartTxn(nil)
   549  
   550  	db3, err := txn3.GetDatabase(name)
   551  	assert.Nil(t, err)
   552  
   553  	rel, err = db3.GetRelationByName(schema.Name)
   554  	assert.Nil(t, err)
   555  	t.Log(rel.String())
   556  }
   557  
   558  func TestTransaction3(t *testing.T) {
   559  	defer testutils.AfterTest(t)()
   560  	ctx := context.Background()
   561  	testutils.EnsureNoLeak(t)
   562  	dir := testutils.InitTestEnv(ModuleName, t)
   563  	c, mgr, driver := initTestContext(ctx, t, dir)
   564  	defer driver.Close()
   565  	defer mgr.Stop()
   566  	defer c.Close()
   567  
   568  	pool, _ := ants.NewPool(20)
   569  	defer pool.Release()
   570  
   571  	var wg sync.WaitGroup
   572  
   573  	flow := func(i int) func() {
   574  		return func() {
   575  			defer wg.Done()
   576  			txn, _ := mgr.StartTxn(nil)
   577  			name := fmt.Sprintf("db-%d", i)
   578  			db, err := txn.CreateDatabase(name, "", "")
   579  			assert.Nil(t, err)
   580  			schema := catalog.MockSchemaAll(13, 12)
   581  			_, err = db.CreateRelation(schema)
   582  			assert.Nil(t, err)
   583  			err = txn.Commit(context.Background())
   584  			assert.Nil(t, err)
   585  		}
   586  	}
   587  
   588  	for i := 0; i < 100; i++ {
   589  		wg.Add(1)
   590  		err := pool.Submit(flow(i))
   591  		assert.Nil(t, err)
   592  	}
   593  	wg.Wait()
   594  }
   595  
   596  func TestObject1(t *testing.T) {
   597  	defer testutils.AfterTest(t)()
   598  	ctx := context.Background()
   599  	testutils.EnsureNoLeak(t)
   600  	dir := testutils.InitTestEnv(ModuleName, t)
   601  	c, mgr, driver := initTestContext(ctx, t, dir)
   602  	defer driver.Close()
   603  	defer mgr.Stop()
   604  	defer c.Close()
   605  
   606  	txn1, _ := mgr.StartTxn(nil)
   607  	name := "db"
   608  	schema := catalog.MockSchema(1, 0)
   609  	db, err := txn1.CreateDatabase(name, "", "")
   610  	assert.Nil(t, err)
   611  	rel, err := db.CreateRelation(schema)
   612  	assert.Nil(t, err)
   613  	_, err = rel.CreateObject(false)
   614  	assert.Nil(t, err)
   615  	err = txn1.Commit(context.Background())
   616  	assert.Nil(t, err)
   617  
   618  	txn2, _ := mgr.StartTxn(nil)
   619  	db, err = txn2.GetDatabase(name)
   620  	assert.Nil(t, err)
   621  	rel, err = db.GetRelationByName(schema.Name)
   622  	assert.Nil(t, err)
   623  	objIt := rel.MakeObjectIt()
   624  	cnt := 0
   625  	for objIt.Valid() {
   626  		iobj := objIt.GetObject()
   627  		t.Log(iobj.String())
   628  		cnt++
   629  		objIt.Next()
   630  	}
   631  	assert.Equal(t, 1, cnt)
   632  
   633  	_, err = rel.CreateObject(false)
   634  	assert.Nil(t, err)
   635  
   636  	objIt = rel.MakeObjectIt()
   637  	cnt = 0
   638  	for objIt.Valid() {
   639  		iobj := objIt.GetObject()
   640  		t.Log(iobj.String())
   641  		cnt++
   642  		objIt.Next()
   643  	}
   644  	assert.Equal(t, 2, cnt)
   645  
   646  	txn3, _ := mgr.StartTxn(nil)
   647  	db, _ = txn3.GetDatabase(name)
   648  	rel, _ = db.GetRelationByName(schema.Name)
   649  	objIt = rel.MakeObjectIt()
   650  	cnt = 0
   651  	for objIt.Valid() {
   652  		iobj := objIt.GetObject()
   653  		t.Log(iobj.String())
   654  		cnt++
   655  		objIt.Next()
   656  	}
   657  	assert.Equal(t, 1, cnt)
   658  
   659  	err = txn2.Commit(context.Background())
   660  	assert.Nil(t, err)
   661  
   662  	objIt = rel.MakeObjectIt()
   663  	cnt = 0
   664  	for objIt.Valid() {
   665  		iobj := objIt.GetObject()
   666  		t.Log(iobj.String())
   667  		cnt++
   668  		objIt.Next()
   669  	}
   670  	assert.Equal(t, 1, cnt)
   671  }
   672  
   673  func TestObject2(t *testing.T) {
   674  	defer testutils.AfterTest(t)()
   675  	ctx := context.Background()
   676  	testutils.EnsureNoLeak(t)
   677  	dir := testutils.InitTestEnv(ModuleName, t)
   678  	c, mgr, driver := initTestContext(ctx, t, dir)
   679  	defer driver.Close()
   680  	defer mgr.Stop()
   681  	defer c.Close()
   682  
   683  	txn1, _ := mgr.StartTxn(nil)
   684  	db, _ := txn1.CreateDatabase("db", "", "")
   685  	schema := catalog.MockSchema(1, 0)
   686  	rel, _ := db.CreateRelation(schema)
   687  	objCnt := 10
   688  	for i := 0; i < objCnt; i++ {
   689  		_, err := rel.CreateObject(false)
   690  		assert.Nil(t, err)
   691  	}
   692  
   693  	it := rel.MakeObjectIt()
   694  	cnt := 0
   695  	for it.Valid() {
   696  		cnt++
   697  		// iobj := it.GetObject()
   698  		it.Next()
   699  	}
   700  	assert.Equal(t, objCnt, cnt)
   701  	// err := txn1.Commit()
   702  	// assert.Nil(t, err)
   703  	t.Log(c.SimplePPString(common.PPL1))
   704  }
   705  
   706  func TestDedup1(t *testing.T) {
   707  	defer testutils.AfterTest(t)()
   708  	ctx := context.Background()
   709  	testutils.EnsureNoLeak(t)
   710  	dir := testutils.InitTestEnv(ModuleName, t)
   711  	c, mgr, driver := initTestContext(ctx, t, dir)
   712  	defer driver.Close()
   713  	defer c.Close()
   714  	defer mgr.Stop()
   715  
   716  	schema := catalog.MockSchemaAll(4, 2)
   717  	schema.BlockMaxRows = 20
   718  	schema.ObjectMaxBlocks = 4
   719  	cnt := uint64(10)
   720  	rows := uint64(schema.BlockMaxRows) / 2 * cnt
   721  	bat := catalog.MockBatch(schema, int(rows))
   722  	defer bat.Close()
   723  	bats := bat.Split(int(cnt))
   724  	{
   725  		txn, _ := mgr.StartTxn(nil)
   726  		db, _ := txn.CreateDatabase("db", "", "")
   727  		_, err := db.CreateRelation(schema)
   728  		assert.Nil(t, err)
   729  		assert.Nil(t, txn.Commit(context.Background()))
   730  	}
   731  	{
   732  		txn, _ := mgr.StartTxn(nil)
   733  		db, _ := txn.GetDatabase("db")
   734  		rel, _ := db.GetRelationByName(schema.Name)
   735  		err := rel.Append(context.Background(), bats[0])
   736  		assert.NoError(t, err)
   737  		err = rel.Append(context.Background(), bats[0])
   738  		assert.True(t, moerr.IsMoErrCode(err, moerr.ErrDuplicateEntry))
   739  		assert.Nil(t, txn.Rollback(context.Background()))
   740  	}
   741  
   742  	{
   743  		txn, _ := mgr.StartTxn(nil)
   744  		db, _ := txn.GetDatabase("db")
   745  		rel, _ := db.GetRelationByName(schema.Name)
   746  		err := rel.Append(context.Background(), bats[0])
   747  		assert.Nil(t, err)
   748  		assert.Nil(t, txn.Commit(context.Background()))
   749  	}
   750  	{
   751  		txn, _ := mgr.StartTxn(nil)
   752  		db, _ := txn.GetDatabase("db")
   753  		rel, _ := db.GetRelationByName(schema.Name)
   754  		err := rel.Append(context.Background(), bats[0])
   755  		assert.True(t, moerr.IsMoErrCode(err, moerr.ErrDuplicateEntry))
   756  		assert.Nil(t, txn.Rollback(context.Background()))
   757  	}
   758  	{
   759  		txn, _ := mgr.StartTxn(nil)
   760  		db, _ := txn.GetDatabase("db")
   761  		rel, _ := db.GetRelationByName(schema.Name)
   762  		err := rel.Append(context.Background(), bats[1])
   763  		assert.Nil(t, err)
   764  
   765  		txn2, _ := mgr.StartTxn(nil)
   766  		db2, _ := txn2.GetDatabase("db")
   767  		rel2, _ := db2.GetRelationByName(schema.Name)
   768  		err = rel2.Append(context.Background(), bats[2])
   769  		assert.Nil(t, err)
   770  		err = rel2.Append(context.Background(), bats[3])
   771  		assert.Nil(t, err)
   772  		assert.Nil(t, txn2.Commit(context.Background()))
   773  
   774  		txn3, _ := mgr.StartTxn(nil)
   775  		db3, _ := txn3.GetDatabase("db")
   776  		rel3, _ := db3.GetRelationByName(schema.Name)
   777  		err = rel3.Append(context.Background(), bats[4])
   778  		assert.Nil(t, err)
   779  		err = rel3.Append(context.Background(), bats[5])
   780  		assert.Nil(t, err)
   781  		assert.Nil(t, txn3.Commit(context.Background()))
   782  
   783  		err = rel.Append(context.Background(), bats[3])
   784  		assert.NoError(t, err)
   785  		err = txn.Commit(context.Background())
   786  		assert.True(t, moerr.IsMoErrCode(err, moerr.ErrTxnWWConflict))
   787  	}
   788  	t.Log(c.SimplePPString(common.PPL1))
   789  }