From fa29368299b3c846ad5ff03cfba88cd35562365b Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 15 Jan 2025 13:41:19 +0530 Subject: [PATCH 01/87] wip --- yb-voyager/cmd/importDataFileBatchProducer.go | 154 ++++++++++++++++++ 1 file changed, 154 insertions(+) create mode 100644 yb-voyager/cmd/importDataFileBatchProducer.go diff --git a/yb-voyager/cmd/importDataFileBatchProducer.go b/yb-voyager/cmd/importDataFileBatchProducer.go new file mode 100644 index 0000000000..ca6335035a --- /dev/null +++ b/yb-voyager/cmd/importDataFileBatchProducer.go @@ -0,0 +1,154 @@ +/* +Copyright (c) YugabyteDB, Inc. + +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 cmd + +import ( + "fmt" + + log "github.com/sirupsen/logrus" + "github.com/yugabyte/yb-voyager/yb-voyager/src/datafile" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" +) + +type FileBatchProducer struct { + task *ImportFileTask + state *ImportDataState + + pendingBatches []*Batch + lastBatchNumber int64 + lastOffset int64 + fileFullySplit bool + completed bool + + datafile *datafile.DataFile + header string +} + +func NewFileBatchProducer(task *ImportFileTask, state *ImportDataState) (*FileBatchProducer, error) { + err := state.PrepareForFileImport(task.FilePath, task.TableNameTup) + if err != nil { + return nil, fmt.Errorf("preparing for file import: %s", err) + } + pendingBatches, lastBatchNumber, lastOffset, fileFullySplit, err := state.Recover(task.FilePath, task.TableNameTup) + if err != nil { + return nil, fmt.Errorf("recovering state for table: %q: %s", task.TableNameTup, err) + } + var completed bool + if len(pendingBatches) == 0 && fileFullySplit { + completed = true + } + + return &FileBatchProducer{ + task: task, + state: state, + pendingBatches: pendingBatches, + lastBatchNumber: lastBatchNumber, + lastOffset: lastOffset, + fileFullySplit: fileFullySplit, + completed: completed, + }, nil +} + +func (p *FileBatchProducer) Done() bool { + return p.completed +} + +func (p *FileBatchProducer) NextBatch() (*Batch, error) { + if len(p.pendingBatches) > 0 { + batch := p.pendingBatches[0] + p.pendingBatches = p.pendingBatches[1:] + // file is fully split and returning the last batch, so mark the producer as completed + if len(p.pendingBatches) == 0 && p.fileFullySplit { + p.completed = true + } + return batch, nil + } + + return p.produceNextBatch() +} + +func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { + if p.datafile == nil { + err := p.openDataFile() + if err != nil { + return nil, err + } + } + + batchWriter, err := p.newBatchWriter() + if err != nil { + return nil, err + } + +} + +func (p *FileBatchProducer) openDataFile() error { + reader, err := dataStore.Open(p.task.FilePath) + if err != nil { + return fmt.Errorf("preparing reader for split generation on file: %q: %v", p.task.FilePath, err) + } + + dataFile, err := datafile.NewDataFile(p.task.FilePath, reader, dataFileDescriptor) + + if err != nil { + return fmt.Errorf("open datafile: %q: %v", p.task.FilePath, err) + } + p.datafile = &dataFile + + log.Infof("Skipping %d lines from %q", p.lastOffset, p.task.FilePath) + err = dataFile.SkipLines(p.lastOffset) + if err != nil { + return fmt.Errorf("skipping line for offset=%d: %v", p.lastOffset, err) + } + if dataFileDescriptor.HasHeader { + p.header = dataFile.GetHeader() + } + return nil +} + +func (p *FileBatchProducer) newBatchWriter() (*BatchWriter, error) { + batchNum := p.lastBatchNumber + 1 + batchWriter := p.state.NewBatchWriter(p.task.FilePath, p.task.TableNameTup, batchNum) + err := batchWriter.Init() + if err != nil { + return nil, fmt.Errorf("initializing batch writer for table: %q: %s", p.task.TableNameTup, err) + } + // Write the header if necessary + if p.header != "" && dataFileDescriptor.FileFormat == datafile.CSV { + err = batchWriter.WriteHeader(p.header) + if err != nil { + utils.ErrExit("writing header for table: %q: %s", p.task.TableNameTup, err) + } + } + return batchWriter, nil +} + +func (p *FileBatchProducer) finalizeBatch(batchWriter *BatchWriter, isLastBatch bool, offsetEnd int64, bytesInBatch int64) (*Batch, error) { + batchNum := p.lastBatchNumber + 1 + batch, err := batchWriter.Done(isLastBatch, offsetEnd, bytesInBatch) + if err != nil { + utils.ErrExit("finalizing batch %d: %s", batchNum, err) + } + batchWriter = nil + p.lastBatchNumber = batchNum + return batch, nil + // submitBatch(batch, updateProgressFn, importBatchArgsProto) + + // Increment batchNum only if this is not the last batch + // if !isLastBatch { + // batchNum++ + // } +} From ec54ea6c27ae2b6585c2861974d6f9283628c75f Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 15 Jan 2025 14:55:55 +0530 Subject: [PATCH 02/87] base logic for producing next batch --- yb-voyager/cmd/importDataFileBatchProducer.go | 74 ++++++++++++++++++- 1 file changed, 71 insertions(+), 3 deletions(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer.go b/yb-voyager/cmd/importDataFileBatchProducer.go index ca6335035a..db21968dbf 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer.go +++ b/yb-voyager/cmd/importDataFileBatchProducer.go @@ -17,6 +17,7 @@ package cmd import ( "fmt" + "io" log "github.com/sirupsen/logrus" "github.com/yugabyte/yb-voyager/yb-voyager/src/datafile" @@ -33,7 +34,7 @@ type FileBatchProducer struct { fileFullySplit bool completed bool - datafile *datafile.DataFile + dataFile datafile.DataFile header string } @@ -81,7 +82,7 @@ func (p *FileBatchProducer) NextBatch() (*Batch, error) { } func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { - if p.datafile == nil { + if p.dataFile == nil { err := p.openDataFile() if err != nil { return nil, err @@ -93,6 +94,73 @@ func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { return nil, err } + var readLineErr error + var line string + var currentBytesRead int64 + batchNum := p.lastBatchNumber + 1 + + for readLineErr == nil { + + line, currentBytesRead, readLineErr = p.dataFile.NextLine() + if readLineErr == nil || (readLineErr == io.EOF && line != "") { + // handling possible case: last dataline(i.e. EOF) but no newline char at the end + numLinesTaken += 1 + } + log.Debugf("Batch %d: totalBytesRead %d, currentBytes %d \n", batchNum, p.dataFile.GetBytesRead(), currentBytesRead) + if currentBytesRead > tdb.MaxBatchSizeInBytes() { + //If a row is itself larger than MaxBatchSizeInBytes erroring out + ybSpecificMsg := "" + if tconf.TargetDBType == YUGABYTEDB { + ybSpecificMsg = ", but should be strictly lower than the the rpc_max_message_size on YugabyteDB (default 267386880 bytes)" + } + utils.ErrExit("record of size %d larger than max batch size: record num=%d for table %q in file %s is larger than the max batch size %d bytes. Max Batch size can be changed using env var MAX_BATCH_SIZE_BYTES%s", currentBytesRead, numLinesTaken, t.ForOutput(), filePath, tdb.MaxBatchSizeInBytes(), ybSpecificMsg) + } + if line != "" { + // can't use importBatchArgsProto.Columns as to use case insenstiive column names + columnNames, _ := TableToColumnNames.Get(t) + line, err = valueConverter.ConvertRow(t, columnNames, line) + if err != nil { + utils.ErrExit("transforming line number=%d for table: %q in file %s: %s", numLinesTaken, p.task.TableNameTup.ForOutput(), p.task.FilePath, err) + } + + // Check if adding this record exceeds the max batch size + if batchWriter.NumRecordsWritten == batchSizeInNumRows || + p.dataFile.GetBytesRead() > tdb.MaxBatchSizeInBytes() { // GetBytesRead - returns the total bytes read until now including the currentBytesRead + + // Finalize the current batch without adding the record + batch, err := p.finalizeBatch(batchWriter, false, numLinesTaken-1, p.dataFile.GetBytesRead()-currentBytesRead) + if err != nil { + return nil, err + } + + //carry forward the bytes to next batch + p.dataFile.ResetBytesRead(currentBytesRead) + + // Start a new batch by calling the initBatchWriter function + initBatchWriter() + } + + // Write the record to the new or current batch + err = batchWriter.WriteRecord(line) + if err != nil { + utils.ErrExit("Write to batch %d: %s", batchNum, err) + } + } + + // Finalize the batch if it's the last line or the end of the file and reset the bytes read to 0 + if readLineErr == io.EOF { + batch, err := p.finalizeBatch(batchWriter, true, numLinesTaken, p.dataFile.GetBytesRead()) + if err != nil { + return nil, err + } + + p.completed = true + p.dataFile.ResetBytesRead(0) + return batch, nil + } else if readLineErr != nil { + return nil, fmt.Errorf("read line from data file: %q: %s", p.task.FilePath, readLineErr) + } + } } func (p *FileBatchProducer) openDataFile() error { @@ -106,7 +174,7 @@ func (p *FileBatchProducer) openDataFile() error { if err != nil { return fmt.Errorf("open datafile: %q: %v", p.task.FilePath, err) } - p.datafile = &dataFile + p.dataFile = dataFile log.Infof("Skipping %d lines from %q", p.lastOffset, p.task.FilePath) err = dataFile.SkipLines(p.lastOffset) From a4aabc8be761f4ccdad169529a2f8f46e6f41a32 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 15 Jan 2025 16:30:53 +0530 Subject: [PATCH 03/87] store line from previous batch --- yb-voyager/cmd/importDataFileBatchProducer.go | 48 ++++++++++++------- 1 file changed, 32 insertions(+), 16 deletions(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer.go b/yb-voyager/cmd/importDataFileBatchProducer.go index db21968dbf..e9872341ab 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer.go +++ b/yb-voyager/cmd/importDataFileBatchProducer.go @@ -34,8 +34,10 @@ type FileBatchProducer struct { fileFullySplit bool completed bool - dataFile datafile.DataFile - header string + dataFile datafile.DataFile + header string + numLinesTaken int64 + lineFromPreviousBatch string } func NewFileBatchProducer(task *ImportFileTask, state *ImportDataState) (*FileBatchProducer, error) { @@ -60,6 +62,7 @@ func NewFileBatchProducer(task *ImportFileTask, state *ImportDataState) (*FileBa lastOffset: lastOffset, fileFullySplit: fileFullySplit, completed: completed, + numLinesTaken: lastOffset, }, nil } @@ -68,6 +71,9 @@ func (p *FileBatchProducer) Done() bool { } func (p *FileBatchProducer) NextBatch() (*Batch, error) { + if p.Done() { + return nil, fmt.Errorf("already done") + } if len(p.pendingBatches) > 0 { batch := p.pendingBatches[0] p.pendingBatches = p.pendingBatches[1:] @@ -89,22 +95,29 @@ func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { } } - batchWriter, err := p.newBatchWriter() - if err != nil { - return nil, err - } - var readLineErr error var line string var currentBytesRead int64 batchNum := p.lastBatchNumber + 1 + batchWriter, err := p.newBatchWriter() + if err != nil { + return nil, err + } + if p.lineFromPreviousBatch != "" { + err = batchWriter.WriteRecord(p.lineFromPreviousBatch) + if err != nil { + return nil, fmt.Errorf("Write to batch %d: %s", batchNum, err) + } + p.lineFromPreviousBatch = "" + } + for readLineErr == nil { line, currentBytesRead, readLineErr = p.dataFile.NextLine() if readLineErr == nil || (readLineErr == io.EOF && line != "") { // handling possible case: last dataline(i.e. EOF) but no newline char at the end - numLinesTaken += 1 + p.numLinesTaken += 1 } log.Debugf("Batch %d: totalBytesRead %d, currentBytes %d \n", batchNum, p.dataFile.GetBytesRead(), currentBytesRead) if currentBytesRead > tdb.MaxBatchSizeInBytes() { @@ -113,14 +126,14 @@ func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { if tconf.TargetDBType == YUGABYTEDB { ybSpecificMsg = ", but should be strictly lower than the the rpc_max_message_size on YugabyteDB (default 267386880 bytes)" } - utils.ErrExit("record of size %d larger than max batch size: record num=%d for table %q in file %s is larger than the max batch size %d bytes. Max Batch size can be changed using env var MAX_BATCH_SIZE_BYTES%s", currentBytesRead, numLinesTaken, t.ForOutput(), filePath, tdb.MaxBatchSizeInBytes(), ybSpecificMsg) + return nil, fmt.Errorf("record of size %d larger than max batch size: record num=%d for table %q in file %s is larger than the max batch size %d bytes. Max Batch size can be changed using env var MAX_BATCH_SIZE_BYTES%s", currentBytesRead, p.numLinesTaken, p.task.TableNameTup.ForOutput(), p.task.FilePath, tdb.MaxBatchSizeInBytes(), ybSpecificMsg) } if line != "" { // can't use importBatchArgsProto.Columns as to use case insenstiive column names - columnNames, _ := TableToColumnNames.Get(t) - line, err = valueConverter.ConvertRow(t, columnNames, line) + columnNames, _ := TableToColumnNames.Get(p.task.TableNameTup) + line, err = valueConverter.ConvertRow(p.task.TableNameTup, columnNames, line) if err != nil { - utils.ErrExit("transforming line number=%d for table: %q in file %s: %s", numLinesTaken, p.task.TableNameTup.ForOutput(), p.task.FilePath, err) + return nil, fmt.Errorf("transforming line number=%d for table: %q in file %s: %s", p.numLinesTaken, p.task.TableNameTup.ForOutput(), p.task.FilePath, err) } // Check if adding this record exceeds the max batch size @@ -128,28 +141,30 @@ func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { p.dataFile.GetBytesRead() > tdb.MaxBatchSizeInBytes() { // GetBytesRead - returns the total bytes read until now including the currentBytesRead // Finalize the current batch without adding the record - batch, err := p.finalizeBatch(batchWriter, false, numLinesTaken-1, p.dataFile.GetBytesRead()-currentBytesRead) + batch, err := p.finalizeBatch(batchWriter, false, p.numLinesTaken-1, p.dataFile.GetBytesRead()-currentBytesRead) if err != nil { return nil, err } //carry forward the bytes to next batch p.dataFile.ResetBytesRead(currentBytesRead) + p.lineFromPreviousBatch = line // Start a new batch by calling the initBatchWriter function - initBatchWriter() + // initBatchWriter() + return batch, nil } // Write the record to the new or current batch err = batchWriter.WriteRecord(line) if err != nil { - utils.ErrExit("Write to batch %d: %s", batchNum, err) + return nil, fmt.Errorf("Write to batch %d: %s", batchNum, err) } } // Finalize the batch if it's the last line or the end of the file and reset the bytes read to 0 if readLineErr == io.EOF { - batch, err := p.finalizeBatch(batchWriter, true, numLinesTaken, p.dataFile.GetBytesRead()) + batch, err := p.finalizeBatch(batchWriter, true, p.numLinesTaken, p.dataFile.GetBytesRead()) if err != nil { return nil, err } @@ -161,6 +176,7 @@ func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { return nil, fmt.Errorf("read line from data file: %q: %s", p.task.FilePath, readLineErr) } } + return nil, fmt.Errorf("unexpected") } func (p *FileBatchProducer) openDataFile() error { From 440a5f3be9ca17374247f2c081ea972d6eacf6b7 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 16 Jan 2025 11:45:52 +0530 Subject: [PATCH 04/87] test --- .../cmd/importDataFileBatchProducer_test.go | 116 ++++++++++++++++++ 1 file changed, 116 insertions(+) create mode 100644 yb-voyager/cmd/importDataFileBatchProducer_test.go diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go new file mode 100644 index 0000000000..76feebc972 --- /dev/null +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -0,0 +1,116 @@ +/* +Copyright (c) YugabyteDB, Inc. + +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 cmd + +import ( + "fmt" + "os" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/yugabyte/yb-voyager/yb-voyager/src/constants" + "github.com/yugabyte/yb-voyager/yb-voyager/src/datafile" + "github.com/yugabyte/yb-voyager/yb-voyager/src/datastore" + "github.com/yugabyte/yb-voyager/yb-voyager/src/dbzm" + "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" +) + +type DummyTdb struct { + tgtdb.TargetYugabyteDB +} + +func (t *DummyTdb) MaxBatchSizeInBytes() int64 { + return 1024 +} + +func createTempFile(dir string, fileContents string) (string, error) { + // Create a temporary file + file, err := os.CreateTemp(dir, "temp-*.txt") + if err != nil { + return "", err + } + defer file.Close() + + // Write some text to the file + _, err = file.WriteString(fileContents) + if err != nil { + return "", err + } + + return file.Name(), nil +} + +func TestBasicFileBatchProducer(t *testing.T) { + fileContents := `id,val +1, "hello"` + + exportDir, err := os.MkdirTemp("/tmp", "export-dir-*") + assert.NoError(t, err) + defer os.RemoveAll(fmt.Sprintf("%s/", exportDir)) + dataDir, err := os.MkdirTemp("/tmp", "data-dir-*") + assert.NoError(t, err) + defer os.RemoveAll(fmt.Sprintf("%s/", dataDir)) + tempFile, err := createTempFile(dataDir, fileContents) + assert.NoError(t, err) + + CreateMigrationProjectIfNotExists(constants.POSTGRESQL, exportDir) + tdb = &DummyTdb{} + valueConverter = &dbzm.NoOpValueConverter{} + dataStore = datastore.NewDataStore(dataDir) + batchSizeInNumRows = 2 + dataFileDescriptor = &datafile.Descriptor{ + FileFormat: "csv", + Delimiter: ",", + HasHeader: true, + ExportDir: exportDir, + QuoteChar: '"', + EscapeChar: '\\', + NullString: "NULL", + // DataFileList: []*FileEntry{ + // { + // FilePath: "file.csv", // Use relative path for testing absolute path handling. + // TableName: "public.my_table", + // RowCount: 100, + // FileSize: 2048, + // }, + // }, + // TableNameToExportedColumns: map[string][]string{ + // "public.my_table": {"id", "name", "age"}, + // }, + } + + sourceName := sqlname.NewObjectName(constants.POSTGRESQL, "public", "public", "test_table") + tableNameTup := sqlname.NameTuple{SourceName: sourceName, CurrentName: sourceName} + task := &ImportFileTask{ + ID: 1, + FilePath: tempFile, + TableNameTup: tableNameTup, + RowCount: 1, + } + + state := NewImportDataState(exportDir) + + batchproducer, err := NewFileBatchProducer(task, state) + assert.NoError(t, err) + + assert.False(t, batchproducer.Done()) + + batch, err := batchproducer.NextBatch() + assert.NoError(t, err) + assert.NotNil(t, batch) + assert.Equal(t, int64(1), batch.RecordCount) +} From c14e69ea6e5e4c449a63ff1c28d0cfdc1c2ed77a Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 16 Jan 2025 12:54:30 +0530 Subject: [PATCH 05/87] rewrite test --- .../cmd/importDataFileBatchProducer_test.go | 76 +++++++++++-------- 1 file changed, 45 insertions(+), 31 deletions(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index 76feebc972..9ddc10a715 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -29,11 +29,11 @@ import ( "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" ) -type DummyTdb struct { +type dummyTDB struct { tgtdb.TargetYugabyteDB } -func (t *DummyTdb) MaxBatchSizeInBytes() int64 { +func (t *dummyTDB) MaxBatchSizeInBytes() int64 { return 1024 } @@ -54,46 +54,44 @@ func createTempFile(dir string, fileContents string) (string, error) { return file.Name(), nil } -func TestBasicFileBatchProducer(t *testing.T) { - fileContents := `id,val -1, "hello"` +func setupDependenciesForTest(batchSize int64) (string, string, *ImportDataState, error) { + lexportDir, err := os.MkdirTemp("/tmp", "export-dir-*") + if err != nil { + return "", "", nil, err + } - exportDir, err := os.MkdirTemp("/tmp", "export-dir-*") - assert.NoError(t, err) - defer os.RemoveAll(fmt.Sprintf("%s/", exportDir)) - dataDir, err := os.MkdirTemp("/tmp", "data-dir-*") - assert.NoError(t, err) - defer os.RemoveAll(fmt.Sprintf("%s/", dataDir)) - tempFile, err := createTempFile(dataDir, fileContents) - assert.NoError(t, err) + ldataDir, err := os.MkdirTemp("/tmp", "data-dir-*") + if err != nil { + return "", "", nil, err + } - CreateMigrationProjectIfNotExists(constants.POSTGRESQL, exportDir) - tdb = &DummyTdb{} + CreateMigrationProjectIfNotExists(constants.POSTGRESQL, lexportDir) + tdb = &dummyTDB{} valueConverter = &dbzm.NoOpValueConverter{} - dataStore = datastore.NewDataStore(dataDir) - batchSizeInNumRows = 2 + dataStore = datastore.NewDataStore(ldataDir) + + batchSizeInNumRows = batchSize + + state := NewImportDataState(lexportDir) + return ldataDir, lexportDir, state, nil +} + +func setupFileForTest(lexportDir string, fileContents string, dir string, tableName string) (string, *ImportFileTask, error) { dataFileDescriptor = &datafile.Descriptor{ FileFormat: "csv", Delimiter: ",", HasHeader: true, - ExportDir: exportDir, + ExportDir: lexportDir, QuoteChar: '"', EscapeChar: '\\', NullString: "NULL", - // DataFileList: []*FileEntry{ - // { - // FilePath: "file.csv", // Use relative path for testing absolute path handling. - // TableName: "public.my_table", - // RowCount: 100, - // FileSize: 2048, - // }, - // }, - // TableNameToExportedColumns: map[string][]string{ - // "public.my_table": {"id", "name", "age"}, - // }, + } + tempFile, err := createTempFile(dir, fileContents) + if err != nil { + return "", nil, err } - sourceName := sqlname.NewObjectName(constants.POSTGRESQL, "public", "public", "test_table") + sourceName := sqlname.NewObjectName(constants.POSTGRESQL, "public", "public", tableName) tableNameTup := sqlname.NameTuple{SourceName: sourceName, CurrentName: sourceName} task := &ImportFileTask{ ID: 1, @@ -101,8 +99,24 @@ func TestBasicFileBatchProducer(t *testing.T) { TableNameTup: tableNameTup, RowCount: 1, } + return tempFile, task, nil +} + +func TestBasicFileBatchProducer(t *testing.T) { + ldataDir, lexportDir, state, err := setupDependenciesForTest(2) + assert.NoError(t, err) - state := NewImportDataState(exportDir) + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + fileContents := `id,val +1, "hello"` + _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) assert.NoError(t, err) From a056385e77f7daeb9f69a47ffec638ff566c7e23 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 16 Jan 2025 12:57:23 +0530 Subject: [PATCH 06/87] minor fix --- yb-voyager/cmd/importDataFileBatchProducer_test.go | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index 9ddc10a715..12d3a6678f 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -30,11 +30,12 @@ import ( ) type dummyTDB struct { + maxSizeBytes int64 tgtdb.TargetYugabyteDB } -func (t *dummyTDB) MaxBatchSizeInBytes() int64 { - return 1024 +func (d *dummyTDB) MaxBatchSizeInBytes() int64 { + return d.maxSizeBytes } func createTempFile(dir string, fileContents string) (string, error) { @@ -54,7 +55,7 @@ func createTempFile(dir string, fileContents string) (string, error) { return file.Name(), nil } -func setupDependenciesForTest(batchSize int64) (string, string, *ImportDataState, error) { +func setupDependenciesForTest(batchSizeRows int64, batchSizeBytes int64) (string, string, *ImportDataState, error) { lexportDir, err := os.MkdirTemp("/tmp", "export-dir-*") if err != nil { return "", "", nil, err @@ -66,11 +67,11 @@ func setupDependenciesForTest(batchSize int64) (string, string, *ImportDataState } CreateMigrationProjectIfNotExists(constants.POSTGRESQL, lexportDir) - tdb = &dummyTDB{} + tdb = &dummyTDB{maxSizeBytes: batchSizeBytes} valueConverter = &dbzm.NoOpValueConverter{} dataStore = datastore.NewDataStore(ldataDir) - batchSizeInNumRows = batchSize + batchSizeInNumRows = batchSizeRows state := NewImportDataState(lexportDir) return ldataDir, lexportDir, state, nil @@ -103,7 +104,7 @@ func setupFileForTest(lexportDir string, fileContents string, dir string, tableN } func TestBasicFileBatchProducer(t *testing.T) { - ldataDir, lexportDir, state, err := setupDependenciesForTest(2) + ldataDir, lexportDir, state, err := setupDependenciesForTest(2, 1024) assert.NoError(t, err) if ldataDir != "" { @@ -127,4 +128,5 @@ func TestBasicFileBatchProducer(t *testing.T) { assert.NoError(t, err) assert.NotNil(t, batch) assert.Equal(t, int64(1), batch.RecordCount) + assert.True(t, batchproducer.Done()) } From c45b6a3213fcee700918285f3320f61bef92cf9e Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 16 Jan 2025 13:11:31 +0530 Subject: [PATCH 07/87] more tests --- .../cmd/importDataFileBatchProducer_test.go | 120 ++++++++++++++++++ 1 file changed, 120 insertions(+) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index 12d3a6678f..be4c4f59d3 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -130,3 +130,123 @@ func TestBasicFileBatchProducer(t *testing.T) { assert.Equal(t, int64(1), batch.RecordCount) assert.True(t, batchproducer.Done()) } + +func TestFileBatchProducerBasedOnRowsThreshold(t *testing.T) { + // max batch size in rows is 2 + ldataDir, lexportDir, state, err := setupDependenciesForTest(2, 1024) + assert.NoError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + fileContents := `id,val +1, "hello" +2, "world" +3, "foo" +4, "bar"` + _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + assert.NoError(t, err) + + batchproducer, err := NewFileBatchProducer(task, state) + assert.NoError(t, err) + + assert.False(t, batchproducer.Done()) + + var batches []*Batch + for !batchproducer.Done() { + batch, err := batchproducer.NextBatch() + assert.NoError(t, err) + assert.NotNil(t, batch) + batches = append(batches, batch) + } + + // 2 batches should be produced + assert.Equal(t, 2, len(batches)) + // each of length 2 + assert.Equal(t, int64(2), batches[0].RecordCount) + assert.Equal(t, int64(2), batches[1].RecordCount) +} + +func TestFileBatchProducerBasedOnSizeThreshold(t *testing.T) { + // max batch size in size is 25 bytes + ldataDir, lexportDir, state, err := setupDependenciesForTest(1000, 25) + assert.NoError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + // each row exccept header is 10 bytes + fileContents := `id,val +1, "abcde" +2, "ghijk" +3, "mnopq" +4, "stuvw"` + _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + assert.NoError(t, err) + + batchproducer, err := NewFileBatchProducer(task, state) + assert.NoError(t, err) + + assert.False(t, batchproducer.Done()) + + var batches []*Batch + for !batchproducer.Done() { + batch, err := batchproducer.NextBatch() + assert.NoError(t, err) + assert.NotNil(t, batch) + batches = append(batches, batch) + } + + // 3 batches should be produced + // while calculating for the first batch, the header is also considered + assert.Equal(t, 3, len(batches)) + // each of length 2 + assert.Equal(t, int64(1), batches[0].RecordCount) + assert.Equal(t, int64(2), batches[1].RecordCount) + assert.Equal(t, int64(1), batches[2].RecordCount) +} + +func TestFileBatchProducerThrowsErrorWhenSingleRowGreaterThanMaxBatchSize(t *testing.T) { + // max batch size in size is 25 bytes + ldataDir, lexportDir, state, err := setupDependenciesForTest(1000, 25) + assert.NoError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + // 3rd row is greater than max batch size + fileContents := `id,val +1, "abcdef" +2, "ghijk" +3, "mnopq1234567899876543" +4, "stuvw"` + _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + assert.NoError(t, err) + + batchproducer, err := NewFileBatchProducer(task, state) + assert.NoError(t, err) + + assert.False(t, batchproducer.Done()) + + // 1st batch is fine. + batch, err := batchproducer.NextBatch() + assert.NoError(t, err) + assert.NotNil(t, batch) + assert.Equal(t, int64(1), batch.RecordCount) + + // 2nd batch should throw error + _, err = batchproducer.NextBatch() + assert.ErrorContains(t, err, "larger than max batch size") +} From a78387ef461f2e0aa3860ff515cec36e9d212441 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 16 Jan 2025 13:17:50 +0530 Subject: [PATCH 08/87] batch value verify --- .../cmd/importDataFileBatchProducer_test.go | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index be4c4f59d3..d9ed8a6bcf 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -168,7 +168,14 @@ func TestFileBatchProducerBasedOnRowsThreshold(t *testing.T) { assert.Equal(t, 2, len(batches)) // each of length 2 assert.Equal(t, int64(2), batches[0].RecordCount) + batchContents, err := os.ReadFile(batches[0].GetFilePath()) + assert.NoError(t, err) + assert.Equal(t, "id,val\n1, \"hello\"\n2, \"world\"", string(batchContents)) + assert.Equal(t, int64(2), batches[1].RecordCount) + batchContents, err = os.ReadFile(batches[1].GetFilePath()) + assert.NoError(t, err) + assert.Equal(t, "id,val\n3, \"foo\"\n4, \"bar\"", string(batchContents)) } func TestFileBatchProducerBasedOnSizeThreshold(t *testing.T) { @@ -210,8 +217,19 @@ func TestFileBatchProducerBasedOnSizeThreshold(t *testing.T) { assert.Equal(t, 3, len(batches)) // each of length 2 assert.Equal(t, int64(1), batches[0].RecordCount) + batchContents, err := os.ReadFile(batches[0].GetFilePath()) + assert.NoError(t, err) + assert.Equal(t, "id,val\n1, \"abcde\"", string(batchContents)) + assert.Equal(t, int64(2), batches[1].RecordCount) + batchContents, err = os.ReadFile(batches[1].GetFilePath()) + assert.NoError(t, err) + assert.Equal(t, "id,val\n2, \"ghijk\"\n3, \"mnopq\"", string(batchContents)) + assert.Equal(t, int64(1), batches[2].RecordCount) + batchContents, err = os.ReadFile(batches[2].GetFilePath()) + assert.NoError(t, err) + assert.Equal(t, "id,val\n4, \"stuvw\"", string(batchContents)) } func TestFileBatchProducerThrowsErrorWhenSingleRowGreaterThanMaxBatchSize(t *testing.T) { From 65c76456b5a57da556001a6c34ceeb0df4ca441f Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 16 Jan 2025 13:19:52 +0530 Subject: [PATCH 09/87] assert less than --- yb-voyager/cmd/importDataFileBatchProducer_test.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index d9ed8a6bcf..8fe9cb4b9b 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -180,7 +180,8 @@ func TestFileBatchProducerBasedOnRowsThreshold(t *testing.T) { func TestFileBatchProducerBasedOnSizeThreshold(t *testing.T) { // max batch size in size is 25 bytes - ldataDir, lexportDir, state, err := setupDependenciesForTest(1000, 25) + maxBatchSizeBytes := int64(25) + ldataDir, lexportDir, state, err := setupDependenciesForTest(1000, maxBatchSizeBytes) assert.NoError(t, err) if ldataDir != "" { @@ -217,16 +218,19 @@ func TestFileBatchProducerBasedOnSizeThreshold(t *testing.T) { assert.Equal(t, 3, len(batches)) // each of length 2 assert.Equal(t, int64(1), batches[0].RecordCount) + assert.LessOrEqual(t, batches[0].ByteCount, maxBatchSizeBytes) batchContents, err := os.ReadFile(batches[0].GetFilePath()) assert.NoError(t, err) assert.Equal(t, "id,val\n1, \"abcde\"", string(batchContents)) assert.Equal(t, int64(2), batches[1].RecordCount) + assert.LessOrEqual(t, batches[1].ByteCount, maxBatchSizeBytes) batchContents, err = os.ReadFile(batches[1].GetFilePath()) assert.NoError(t, err) assert.Equal(t, "id,val\n2, \"ghijk\"\n3, \"mnopq\"", string(batchContents)) assert.Equal(t, int64(1), batches[2].RecordCount) + assert.LessOrEqual(t, batches[2].ByteCount, maxBatchSizeBytes) batchContents, err = os.ReadFile(batches[2].GetFilePath()) assert.NoError(t, err) assert.Equal(t, "id,val\n4, \"stuvw\"", string(batchContents)) From a9a294c832fad8feee0039ef081a8a966f0a741a Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 16 Jan 2025 13:33:16 +0530 Subject: [PATCH 10/87] resumable test --- .../cmd/importDataFileBatchProducer_test.go | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index 8fe9cb4b9b..ed535c7497 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -20,6 +20,7 @@ import ( "os" "testing" + "github.com/google/go-cmp/cmp" "github.com/stretchr/testify/assert" "github.com/yugabyte/yb-voyager/yb-voyager/src/constants" "github.com/yugabyte/yb-voyager/yb-voyager/src/datafile" @@ -272,3 +273,59 @@ func TestFileBatchProducerThrowsErrorWhenSingleRowGreaterThanMaxBatchSize(t *tes _, err = batchproducer.NextBatch() assert.ErrorContains(t, err, "larger than max batch size") } + +func TestFileBatchProducerResumable(t *testing.T) { + // max batch size in rows is 2 + ldataDir, lexportDir, state, err := setupDependenciesForTest(2, 1024) + assert.NoError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + fileContents := `id,val +1, "hello" +2, "world" +3, "foo" +4, "bar"` + _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + assert.NoError(t, err) + + batchproducer, err := NewFileBatchProducer(task, state) + assert.NoError(t, err) + assert.False(t, batchproducer.Done()) + + // generate one batch + batch1, err := batchproducer.NextBatch() + assert.NoError(t, err) + assert.NotNil(t, batch1) + assert.Equal(t, int64(2), batch1.RecordCount) + + // simulate a crash and recover + batchproducer, err = NewFileBatchProducer(task, state) + assert.NoError(t, err) + assert.False(t, batchproducer.Done()) + + // state should have recovered that one batch + assert.Equal(t, 1, len(batchproducer.pendingBatches)) + assert.True(t, cmp.Equal(batch1, batchproducer.pendingBatches[0])) + + // verify that it picks up from pendingBatches + // instead of procing a new batch. + batch1Recovered, err := batchproducer.NextBatch() + assert.NoError(t, err) + assert.NotNil(t, batch1Recovered) + assert.True(t, cmp.Equal(batch1, batch1Recovered)) + assert.Equal(t, 0, len(batchproducer.pendingBatches)) + assert.False(t, batchproducer.Done()) + + // get final batch + batch2, err := batchproducer.NextBatch() + assert.NoError(t, err) + assert.NotNil(t, batch2) + assert.Equal(t, int64(2), batch2.RecordCount) + assert.True(t, batchproducer.Done()) +} From 1f32f7a6938f1647803d3315115ffbdda818d406 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 16 Jan 2025 13:44:53 +0530 Subject: [PATCH 11/87] import data change to use filebatchproducer --- yb-voyager/cmd/importData.go | 27 ++++++++++++++++++++------- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index 45050cf615..1d44f4d0f7 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -1010,17 +1010,30 @@ func importFile(state *ImportDataState, task *ImportFileTask, updateProgressFn f if err != nil { utils.ErrExit("preparing for file import: %s", err) } - log.Infof("Collect all interrupted/remaining splits.") - pendingBatches, lastBatchNumber, lastOffset, fileFullySplit, err := state.Recover(task.FilePath, task.TableNameTup) + + fileBatchProducer, err := NewFileBatchProducer(task, state) if err != nil { - utils.ErrExit("recovering state for table: %q: %s", task.TableNameTup, err) + utils.ErrExit("creating file batch producer: %s", err) } - for _, batch := range pendingBatches { + + for !fileBatchProducer.Done() { + batch, err := fileBatchProducer.NextBatch() + if err != nil { + utils.ErrExit("getting next batch: %s", err) + } submitBatch(batch, updateProgressFn, importBatchArgsProto) } - if !fileFullySplit { - splitFilesForTable(state, origDataFile, task.TableNameTup, lastBatchNumber, lastOffset, updateProgressFn, importBatchArgsProto) - } + // log.Infof("Collect all interrupted/remaining splits.") + // pendingBatches, lastBatchNumber, lastOffset, fileFullySplit, err := state.Recover(task.FilePath, task.TableNameTup) + // if err != nil { + // utils.ErrExit("recovering state for table: %q: %s", task.TableNameTup, err) + // } + // for _, batch := range pendingBatches { + // submitBatch(batch, updateProgressFn, importBatchArgsProto) + // } + // if !fileFullySplit { + // splitFilesForTable(state, origDataFile, task.TableNameTup, lastBatchNumber, lastOffset, updateProgressFn, importBatchArgsProto) + // } } func splitFilesForTable(state *ImportDataState, filePath string, t sqlname.NameTuple, From cbc1ede3c7fac201111b00fe752bb959748b934a Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 16 Jan 2025 13:54:10 +0530 Subject: [PATCH 12/87] unit tag --- yb-voyager/cmd/importDataFileBatchProducer_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index ed535c7497..852dccee60 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -1,3 +1,5 @@ +//go:build unit + /* Copyright (c) YugabyteDB, Inc. From 943bd22b6d946ddc219ffe7bb7014e3ab52dfb82 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 20 Jan 2025 11:13:32 +0530 Subject: [PATCH 13/87] cleanup --- yb-voyager/cmd/importData.go | 136 ------------------ yb-voyager/cmd/importDataFileBatchProducer.go | 11 +- 2 files changed, 3 insertions(+), 144 deletions(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index 1d44f4d0f7..09a48e01d8 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -17,7 +17,6 @@ package cmd import ( "fmt" - "io" "os" "os/exec" "path/filepath" @@ -1023,141 +1022,6 @@ func importFile(state *ImportDataState, task *ImportFileTask, updateProgressFn f } submitBatch(batch, updateProgressFn, importBatchArgsProto) } - // log.Infof("Collect all interrupted/remaining splits.") - // pendingBatches, lastBatchNumber, lastOffset, fileFullySplit, err := state.Recover(task.FilePath, task.TableNameTup) - // if err != nil { - // utils.ErrExit("recovering state for table: %q: %s", task.TableNameTup, err) - // } - // for _, batch := range pendingBatches { - // submitBatch(batch, updateProgressFn, importBatchArgsProto) - // } - // if !fileFullySplit { - // splitFilesForTable(state, origDataFile, task.TableNameTup, lastBatchNumber, lastOffset, updateProgressFn, importBatchArgsProto) - // } -} - -func splitFilesForTable(state *ImportDataState, filePath string, t sqlname.NameTuple, - lastBatchNumber int64, lastOffset int64, updateProgressFn func(int64), importBatchArgsProto *tgtdb.ImportBatchArgs) { - log.Infof("Split data file %q: tableName=%q, largestSplit=%v, largestOffset=%v", filePath, t, lastBatchNumber, lastOffset) - batchNum := lastBatchNumber + 1 - numLinesTaken := lastOffset - - reader, err := dataStore.Open(filePath) - if err != nil { - utils.ErrExit("preparing reader for split generation on file: %q: %v", filePath, err) - } - - dataFile, err := datafile.NewDataFile(filePath, reader, dataFileDescriptor) - if err != nil { - utils.ErrExit("open datafile: %q: %v", filePath, err) - } - defer dataFile.Close() - - log.Infof("Skipping %d lines from %q", lastOffset, filePath) - err = dataFile.SkipLines(lastOffset) - if err != nil { - utils.ErrExit("skipping line for offset=%d: %v", lastOffset, err) - } - - var readLineErr error = nil - var line string - var currentBytesRead int64 - var batchWriter *BatchWriter - header := "" - if dataFileDescriptor.HasHeader { - header = dataFile.GetHeader() - } - - // Helper function to initialize a new batchWriter - initBatchWriter := func() { - batchWriter = state.NewBatchWriter(filePath, t, batchNum) - err := batchWriter.Init() - if err != nil { - utils.ErrExit("initializing batch writer for table: %q: %s", t, err) - } - // Write the header if necessary - if header != "" && dataFileDescriptor.FileFormat == datafile.CSV { - err = batchWriter.WriteHeader(header) - if err != nil { - utils.ErrExit("writing header for table: %q: %s", t, err) - } - } - } - - // Function to finalize and submit the current batch - finalizeBatch := func(isLastBatch bool, offsetEnd int64, bytesInBatch int64) { - batch, err := batchWriter.Done(isLastBatch, offsetEnd, bytesInBatch) - if err != nil { - utils.ErrExit("finalizing batch %d: %s", batchNum, err) - } - batchWriter = nil - submitBatch(batch, updateProgressFn, importBatchArgsProto) - - // Increment batchNum only if this is not the last batch - if !isLastBatch { - batchNum++ - } - } - - for readLineErr == nil { - - if batchWriter == nil { - initBatchWriter() // Create a new batchWriter - } - - line, currentBytesRead, readLineErr = dataFile.NextLine() - if readLineErr == nil || (readLineErr == io.EOF && line != "") { - // handling possible case: last dataline(i.e. EOF) but no newline char at the end - numLinesTaken += 1 - } - log.Debugf("Batch %d: totalBytesRead %d, currentBytes %d \n", batchNum, dataFile.GetBytesRead(), currentBytesRead) - if currentBytesRead > tdb.MaxBatchSizeInBytes() { - //If a row is itself larger than MaxBatchSizeInBytes erroring out - ybSpecificMsg := "" - if tconf.TargetDBType == YUGABYTEDB { - ybSpecificMsg = ", but should be strictly lower than the the rpc_max_message_size on YugabyteDB (default 267386880 bytes)" - } - utils.ErrExit("record of size %d larger than max batch size: record num=%d for table %q in file %s is larger than the max batch size %d bytes. Max Batch size can be changed using env var MAX_BATCH_SIZE_BYTES%s", currentBytesRead, numLinesTaken, t.ForOutput(), filePath, tdb.MaxBatchSizeInBytes(), ybSpecificMsg) - } - if line != "" { - // can't use importBatchArgsProto.Columns as to use case insenstiive column names - columnNames, _ := TableToColumnNames.Get(t) - line, err = valueConverter.ConvertRow(t, columnNames, line) - if err != nil { - utils.ErrExit("transforming line number=%d for table: %q in file %s: %s", numLinesTaken, t.ForOutput(), filePath, err) - } - - // Check if adding this record exceeds the max batch size - if batchWriter.NumRecordsWritten == batchSizeInNumRows || - dataFile.GetBytesRead() > tdb.MaxBatchSizeInBytes() { // GetBytesRead - returns the total bytes read until now including the currentBytesRead - - // Finalize the current batch without adding the record - finalizeBatch(false, numLinesTaken-1, dataFile.GetBytesRead()-currentBytesRead) - - //carry forward the bytes to next batch - dataFile.ResetBytesRead(currentBytesRead) - - // Start a new batch by calling the initBatchWriter function - initBatchWriter() - } - - // Write the record to the new or current batch - err = batchWriter.WriteRecord(line) - if err != nil { - utils.ErrExit("Write to batch %d: %s", batchNum, err) - } - } - - // Finalize the batch if it's the last line or the end of the file and reset the bytes read to 0 - if readLineErr == io.EOF { - finalizeBatch(true, numLinesTaken, dataFile.GetBytesRead()) - dataFile.ResetBytesRead(0) - } else if readLineErr != nil { - utils.ErrExit("read line from data file: %q: %s", filePath, readLineErr) - } - } - - log.Infof("splitFilesForTable: done splitting data file %q for table %q", filePath, t) } func executePostSnapshotImportSqls() error { diff --git a/yb-voyager/cmd/importDataFileBatchProducer.go b/yb-voyager/cmd/importDataFileBatchProducer.go index e9872341ab..4bccbd53f2 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer.go +++ b/yb-voyager/cmd/importDataFileBatchProducer.go @@ -72,7 +72,7 @@ func (p *FileBatchProducer) Done() bool { func (p *FileBatchProducer) NextBatch() (*Batch, error) { if p.Done() { - return nil, fmt.Errorf("already done") + return nil, fmt.Errorf("already completed producing all batches") } if len(p.pendingBatches) > 0 { batch := p.pendingBatches[0] @@ -176,7 +176,8 @@ func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { return nil, fmt.Errorf("read line from data file: %q: %s", p.task.FilePath, readLineErr) } } - return nil, fmt.Errorf("unexpected") + // ideally should not reach here + return nil, fmt.Errorf("could not produce next batch: err: %w", readLineErr) } func (p *FileBatchProducer) openDataFile() error { @@ -229,10 +230,4 @@ func (p *FileBatchProducer) finalizeBatch(batchWriter *BatchWriter, isLastBatch batchWriter = nil p.lastBatchNumber = batchNum return batch, nil - // submitBatch(batch, updateProgressFn, importBatchArgsProto) - - // Increment batchNum only if this is not the last batch - // if !isLastBatch { - // batchNum++ - // } } From c8390e3d2291f08d28776ac85318168a4f7b9bdf Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 21 Jan 2025 13:55:01 +0530 Subject: [PATCH 14/87] wip --- yb-voyager/cmd/importDataFileTaskImporter.go | 25 ++++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 yb-voyager/cmd/importDataFileTaskImporter.go diff --git a/yb-voyager/cmd/importDataFileTaskImporter.go b/yb-voyager/cmd/importDataFileTaskImporter.go new file mode 100644 index 0000000000..af9d1c3f97 --- /dev/null +++ b/yb-voyager/cmd/importDataFileTaskImporter.go @@ -0,0 +1,25 @@ +/* +Copyright (c) YugabyteDB, Inc. + +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 cmd + +import "github.com/sourcegraph/conc/pool" + +type FileTaskImporter struct { + task *ImportFileTask + state *ImportDataState + batchProducer *FileBatchProducer + workerPool *pool.Pool +} From 3571226561ba1609dee909f156dfd8d887a566c9 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 22 Jan 2025 10:38:35 +0530 Subject: [PATCH 15/87] filetaskimporter --- yb-voyager/cmd/importData.go | 113 --------- yb-voyager/cmd/importDataFileTaskImporter.go | 232 ++++++++++++++++++- 2 files changed, 227 insertions(+), 118 deletions(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index 09a48e01d8..b58c616935 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -855,30 +855,6 @@ func getIdentityColumnsForTables(tables []sqlname.NameTuple, identityType string return result } -func getTotalProgressAmount(task *ImportFileTask) int64 { - if reportProgressInBytes { - return task.FileSize - } else { - return task.RowCount - } -} - -func getImportedProgressAmount(task *ImportFileTask, state *ImportDataState) int64 { - if reportProgressInBytes { - byteCount, err := state.GetImportedByteCount(task.FilePath, task.TableNameTup) - if err != nil { - utils.ErrExit("Failed to get imported byte count for table: %s: %s", task.TableNameTup, err) - } - return byteCount - } else { - rowCount, err := state.GetImportedRowCount(task.FilePath, task.TableNameTup) - if err != nil { - utils.ErrExit("Failed to get imported row count for table: %s: %s", task.TableNameTup, err) - } - return rowCount - } -} - func importFileTasksToTableNames(tasks []*ImportFileTask) []string { tableNames := []string{} for _, t := range tasks { @@ -974,31 +950,6 @@ func cleanImportState(state *ImportDataState, tasks []*ImportFileTask) { } } -func getImportBatchArgsProto(tableNameTup sqlname.NameTuple, filePath string) *tgtdb.ImportBatchArgs { - columns, _ := TableToColumnNames.Get(tableNameTup) - columns, err := tdb.QuoteAttributeNames(tableNameTup, columns) - if err != nil { - utils.ErrExit("if required quote column names: %s", err) - } - // If `columns` is unset at this point, no attribute list is passed in the COPY command. - fileFormat := dataFileDescriptor.FileFormat - if fileFormat == datafile.SQL { - fileFormat = datafile.TEXT - } - importBatchArgsProto := &tgtdb.ImportBatchArgs{ - TableNameTup: tableNameTup, - Columns: columns, - FileFormat: fileFormat, - Delimiter: dataFileDescriptor.Delimiter, - HasHeader: dataFileDescriptor.HasHeader && fileFormat == datafile.CSV, - QuoteChar: dataFileDescriptor.QuoteChar, - EscapeChar: dataFileDescriptor.EscapeChar, - NullString: dataFileDescriptor.NullString, - } - log.Infof("ImportBatchArgs: %v", spew.Sdump(importBatchArgsProto)) - return importBatchArgsProto -} - func importFile(state *ImportDataState, task *ImportFileTask, updateProgressFn func(int64)) { origDataFile := task.FilePath @@ -1051,44 +1002,6 @@ func submitBatch(batch *Batch, updateProgressFn func(int64), importBatchArgsProt log.Infof("Queued batch: %s", spew.Sdump(batch)) } -func importBatch(batch *Batch, importBatchArgsProto *tgtdb.ImportBatchArgs) { - err := batch.MarkPending() - if err != nil { - utils.ErrExit("marking batch as pending: %d: %s", batch.Number, err) - } - log.Infof("Importing %q", batch.FilePath) - - importBatchArgs := *importBatchArgsProto - importBatchArgs.FilePath = batch.FilePath - importBatchArgs.RowsPerTransaction = batch.OffsetEnd - batch.OffsetStart - - var rowsAffected int64 - sleepIntervalSec := 0 - for attempt := 0; attempt < COPY_MAX_RETRY_COUNT; attempt++ { - tableSchema, _ := TableNameToSchema.Get(batch.TableNameTup) - rowsAffected, err = tdb.ImportBatch(batch, &importBatchArgs, exportDir, tableSchema) - if err == nil || tdb.IsNonRetryableCopyError(err) { - break - } - log.Warnf("COPY FROM file %q: %s", batch.FilePath, err) - sleepIntervalSec += 10 - if sleepIntervalSec > MAX_SLEEP_SECOND { - sleepIntervalSec = MAX_SLEEP_SECOND - } - log.Infof("sleep for %d seconds before retrying the file %s (attempt %d)", - sleepIntervalSec, batch.FilePath, attempt) - time.Sleep(time.Duration(sleepIntervalSec) * time.Second) - } - log.Infof("%q => %d rows affected", batch.FilePath, rowsAffected) - if err != nil { - utils.ErrExit("import batch: %q into %s: %s", batch.FilePath, batch.TableNameTup, err) - } - err = batch.MarkDone() - if err != nil { - utils.ErrExit("marking batch as done: %q: %s", batch.FilePath, err) - } -} - func getIndexName(sqlQuery string, indexName string) (string, error) { // Return the index name itself if it is aleady qualified with schema name if len(strings.Split(indexName, ".")) == 2 { @@ -1237,29 +1150,3 @@ func createInitialImportDataTableMetrics(tasks []*ImportFileTask) []*cp.UpdateIm return result } - -func createImportDataTableMetrics(tableName string, countLiveRows int64, countTotalRows int64, - status int) cp.UpdateImportedRowCountEvent { - - var schemaName, tableName2 string - if strings.Count(tableName, ".") == 1 { - schemaName, tableName2 = cp.SplitTableNameForPG(tableName) - } else { - schemaName, tableName2 = tconf.Schema, tableName - } - result := cp.UpdateImportedRowCountEvent{ - BaseUpdateRowCountEvent: cp.BaseUpdateRowCountEvent{ - BaseEvent: cp.BaseEvent{ - EventType: "IMPORT DATA", - MigrationUUID: migrationUUID, - SchemaNames: []string{schemaName}, - }, - TableName: tableName2, - Status: cp.EXPORT_OR_IMPORT_DATA_STATUS_INT_TO_STR[status], - TotalRowCount: countTotalRows, - CompletedRowCount: countLiveRows, - }, - } - - return result -} diff --git a/yb-voyager/cmd/importDataFileTaskImporter.go b/yb-voyager/cmd/importDataFileTaskImporter.go index af9d1c3f97..2b5834e3b8 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter.go +++ b/yb-voyager/cmd/importDataFileTaskImporter.go @@ -15,11 +15,233 @@ limitations under the License. */ package cmd -import "github.com/sourcegraph/conc/pool" +import ( + "fmt" + "strings" + "time" + + "github.com/davecgh/go-spew/spew" + log "github.com/sirupsen/logrus" + "github.com/sourcegraph/conc/pool" + "github.com/yugabyte/yb-voyager/yb-voyager/src/cp" + "github.com/yugabyte/yb-voyager/yb-voyager/src/datafile" + "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" +) type FileTaskImporter struct { - task *ImportFileTask - state *ImportDataState - batchProducer *FileBatchProducer - workerPool *pool.Pool + task *ImportFileTask + state *ImportDataState + batchProducer *FileBatchProducer + importBatchArgsProto *tgtdb.ImportBatchArgs + workerPool *pool.Pool + + totalProgressAmount int64 + currentProgressAmount int64 + progressReporter *ImportDataProgressReporter +} + +func NewFileTaskImporter(task *ImportFileTask, state *ImportDataState, workerPool *pool.Pool, + progressReporter *ImportDataProgressReporter) (*FileTaskImporter, error) { + batchProducer, err := NewFileBatchProducer(task, state) + if err != nil { + return nil, fmt.Errorf("creating file batch producer: %s", err) + } + totalProgressAmount := getTotalProgressAmount(task) + progressReporter.ImportFileStarted(task, totalProgressAmount) + progressReporter.AddProgressAmount(task, getImportedProgressAmount(task, state)) + + fti := &FileTaskImporter{ + task: task, + state: state, + batchProducer: batchProducer, + workerPool: workerPool, + importBatchArgsProto: getImportBatchArgsProto(task.TableNameTup, task.FilePath), + progressReporter: progressReporter, + totalProgressAmount: getTotalProgressAmount(task), + } + return fti, nil +} + +// as of now, batch production and batch submission +// is done together in `SubmitNextBatch` method. +// In other words, as soon as a batch is produced, it is submitted. +// Therefore, to check whether all batches are submitted, we can check +// if the batch producer is done. +func (fti *FileTaskImporter) AllBatchesSubmitted() bool { + return fti.batchProducer.Done() +} + +func (fti *FileTaskImporter) AllBatchesImported() error { + // TODO: check importDataState for status. + panic("not implemented") +} + +func (fti *FileTaskImporter) SubmitNextBatch() error { + if fti.batchProducer.Done() { + return fmt.Errorf("no more batches to submit") + } + batch, err := fti.batchProducer.NextBatch() + if err != nil { + return fmt.Errorf("getting next batch: %w", err) + } + return fti.submitBatch(batch) +} + +func (fti *FileTaskImporter) submitBatch(batch *Batch) error { + fti.workerPool.Go(func() { + // There are `poolSize` number of competing go-routines trying to invoke COPY. + // But the `connPool` will allow only `parallelism` number of connections to be + // used at a time. Thus limiting the number of concurrent COPYs to `parallelism`. + importBatch(batch, fti.importBatchArgsProto) + if reportProgressInBytes { + fti.updateProgress(batch.ByteCount) + } else { + fti.updateProgress(batch.RecordCount) + } + }) + log.Infof("Queued batch: %s", spew.Sdump(batch)) + return nil +} + +func (fti *FileTaskImporter) updateProgress(progressAmount int64) { + fti.currentProgressAmount += progressAmount + fti.progressReporter.AddProgressAmount(fti.task, progressAmount) + + if importerRole == TARGET_DB_IMPORTER_ROLE && fti.totalProgressAmount > fti.currentProgressAmount { + importDataTableMetrics := createImportDataTableMetrics(fti.task.TableNameTup.ForKey(), + fti.currentProgressAmount, fti.totalProgressAmount, ROW_UPDATE_STATUS_IN_PROGRESS) + // The metrics are sent after evry 5 secs in implementation of UpdateImportedRowCount + controlPlane.UpdateImportedRowCount( + []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) + } +} + +func (fti *FileTaskImporter) PostProcess() { + // TODO: close fileBatchProducer properly + + if importerRole == TARGET_DB_IMPORTER_ROLE { + importDataTableMetrics := createImportDataTableMetrics(fti.task.TableNameTup.ForKey(), + fti.currentProgressAmount, fti.totalProgressAmount, ROW_UPDATE_STATUS_COMPLETED) + controlPlane.UpdateImportedRowCount( + []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) + } + + fti.progressReporter.FileImportDone(fti.task) // Remove the progress-bar for the file.\ +} + +// ============================================================================= // + +func getTotalProgressAmount(task *ImportFileTask) int64 { + if reportProgressInBytes { + return task.FileSize + } else { + return task.RowCount + } +} + +func getImportedProgressAmount(task *ImportFileTask, state *ImportDataState) int64 { + if reportProgressInBytes { + byteCount, err := state.GetImportedByteCount(task.FilePath, task.TableNameTup) + if err != nil { + utils.ErrExit("Failed to get imported byte count for table: %s: %s", task.TableNameTup, err) + } + return byteCount + } else { + rowCount, err := state.GetImportedRowCount(task.FilePath, task.TableNameTup) + if err != nil { + utils.ErrExit("Failed to get imported row count for table: %s: %s", task.TableNameTup, err) + } + return rowCount + } +} + +func createImportDataTableMetrics(tableName string, countLiveRows int64, countTotalRows int64, + status int) cp.UpdateImportedRowCountEvent { + + var schemaName, tableName2 string + if strings.Count(tableName, ".") == 1 { + schemaName, tableName2 = cp.SplitTableNameForPG(tableName) + } else { + schemaName, tableName2 = tconf.Schema, tableName + } + result := cp.UpdateImportedRowCountEvent{ + BaseUpdateRowCountEvent: cp.BaseUpdateRowCountEvent{ + BaseEvent: cp.BaseEvent{ + EventType: "IMPORT DATA", + MigrationUUID: migrationUUID, + SchemaNames: []string{schemaName}, + }, + TableName: tableName2, + Status: cp.EXPORT_OR_IMPORT_DATA_STATUS_INT_TO_STR[status], + TotalRowCount: countTotalRows, + CompletedRowCount: countLiveRows, + }, + } + + return result +} + +func getImportBatchArgsProto(tableNameTup sqlname.NameTuple, filePath string) *tgtdb.ImportBatchArgs { + columns, _ := TableToColumnNames.Get(tableNameTup) + columns, err := tdb.QuoteAttributeNames(tableNameTup, columns) + if err != nil { + utils.ErrExit("if required quote column names: %s", err) + } + // If `columns` is unset at this point, no attribute list is passed in the COPY command. + fileFormat := dataFileDescriptor.FileFormat + if fileFormat == datafile.SQL { + fileFormat = datafile.TEXT + } + importBatchArgsProto := &tgtdb.ImportBatchArgs{ + TableNameTup: tableNameTup, + Columns: columns, + FileFormat: fileFormat, + Delimiter: dataFileDescriptor.Delimiter, + HasHeader: dataFileDescriptor.HasHeader && fileFormat == datafile.CSV, + QuoteChar: dataFileDescriptor.QuoteChar, + EscapeChar: dataFileDescriptor.EscapeChar, + NullString: dataFileDescriptor.NullString, + } + log.Infof("ImportBatchArgs: %v", spew.Sdump(importBatchArgsProto)) + return importBatchArgsProto +} + +func importBatch(batch *Batch, importBatchArgsProto *tgtdb.ImportBatchArgs) { + err := batch.MarkPending() + if err != nil { + utils.ErrExit("marking batch as pending: %d: %s", batch.Number, err) + } + log.Infof("Importing %q", batch.FilePath) + + importBatchArgs := *importBatchArgsProto + importBatchArgs.FilePath = batch.FilePath + importBatchArgs.RowsPerTransaction = batch.OffsetEnd - batch.OffsetStart + + var rowsAffected int64 + sleepIntervalSec := 0 + for attempt := 0; attempt < COPY_MAX_RETRY_COUNT; attempt++ { + tableSchema, _ := TableNameToSchema.Get(batch.TableNameTup) + rowsAffected, err = tdb.ImportBatch(batch, &importBatchArgs, exportDir, tableSchema) + if err == nil || tdb.IsNonRetryableCopyError(err) { + break + } + log.Warnf("COPY FROM file %q: %s", batch.FilePath, err) + sleepIntervalSec += 10 + if sleepIntervalSec > MAX_SLEEP_SECOND { + sleepIntervalSec = MAX_SLEEP_SECOND + } + log.Infof("sleep for %d seconds before retrying the file %s (attempt %d)", + sleepIntervalSec, batch.FilePath, attempt) + time.Sleep(time.Duration(sleepIntervalSec) * time.Second) + } + log.Infof("%q => %d rows affected", batch.FilePath, rowsAffected) + if err != nil { + utils.ErrExit("import batch: %q into %s: %s", batch.FilePath, batch.TableNameTup, err) + } + err = batch.MarkDone() + if err != nil { + utils.ErrExit("marking batch as done: %q: %s", batch.FilePath, err) + } } From 1e4b1c8dcb3a94d353253aebd02b4bbc499af197 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 22 Jan 2025 10:39:48 +0530 Subject: [PATCH 16/87] run tests --- .github/workflows/go.yml | 2 +- .github/workflows/integration-tests.yml | 2 +- .github/workflows/issue-tests.yml | 2 +- .github/workflows/misc-migtests.yml | 2 +- .github/workflows/mysql-migtests.yml | 2 +- .github/workflows/pg-13-migtests.yml | 2 +- .github/workflows/pg-17-migtests.yml | 2 +- .github/workflows/pg-9-migtests.yml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index 88c20428d5..2d6a1c8098 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: [main] + branches: ['*'] jobs: diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index d94a594cb1..4d71e66fd3 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: [main] + branches: ['*'] env: ORACLE_INSTANT_CLIENT_VERSION: "21.5.0.0.0-1" diff --git a/.github/workflows/issue-tests.yml b/.github/workflows/issue-tests.yml index 550eef3c8a..3b72d282fd 100644 --- a/.github/workflows/issue-tests.yml +++ b/.github/workflows/issue-tests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: [main] + branches: ['*'] jobs: diff --git a/.github/workflows/misc-migtests.yml b/.github/workflows/misc-migtests.yml index 3362028481..fdb8fe0e1e 100644 --- a/.github/workflows/misc-migtests.yml +++ b/.github/workflows/misc-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-misc-migration-tests: diff --git a/.github/workflows/mysql-migtests.yml b/.github/workflows/mysql-migtests.yml index 1dfda76a62..65e15e2431 100644 --- a/.github/workflows/mysql-migtests.yml +++ b/.github/workflows/mysql-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-mysql-migration-tests: diff --git a/.github/workflows/pg-13-migtests.yml b/.github/workflows/pg-13-migtests.yml index d6b03ec165..43762a0fa6 100644 --- a/.github/workflows/pg-13-migtests.yml +++ b/.github/workflows/pg-13-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-pg-13-migration-tests: diff --git a/.github/workflows/pg-17-migtests.yml b/.github/workflows/pg-17-migtests.yml index 27aacc0c68..231ec42282 100644 --- a/.github/workflows/pg-17-migtests.yml +++ b/.github/workflows/pg-17-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-pg-17-migration-tests: diff --git a/.github/workflows/pg-9-migtests.yml b/.github/workflows/pg-9-migtests.yml index 8fa3251d1b..44c0e2a81f 100644 --- a/.github/workflows/pg-9-migtests.yml +++ b/.github/workflows/pg-9-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-pg-9-migration-tests: From 0c2ec2476d228a5715693a624f44b68d132a362d Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 22 Jan 2025 10:45:35 +0530 Subject: [PATCH 17/87] use task importer --- yb-voyager/cmd/importData.go | 132 +++++++++++++++++++---------------- 1 file changed, 72 insertions(+), 60 deletions(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index b58c616935..e15d650ebc 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -591,36 +591,48 @@ func importData(importFileTasks []*ImportFileTask) { batchImportPool = pool.New().WithMaxGoroutines(poolSize) log.Infof("created batch import pool of size: %d", poolSize) - totalProgressAmount := getTotalProgressAmount(task) - progressReporter.ImportFileStarted(task, totalProgressAmount) - importedProgressAmount := getImportedProgressAmount(task, state) - progressReporter.AddProgressAmount(task, importedProgressAmount) - - var currentProgress int64 - updateProgressFn := func(progressAmount int64) { - currentProgress += progressAmount - progressReporter.AddProgressAmount(task, progressAmount) - - if importerRole == TARGET_DB_IMPORTER_ROLE && totalProgressAmount > currentProgress { - importDataTableMetrics := createImportDataTableMetrics(task.TableNameTup.ForKey(), - currentProgress, totalProgressAmount, ROW_UPDATE_STATUS_IN_PROGRESS) - // The metrics are sent after evry 5 secs in implementation of UpdateImportedRowCount - controlPlane.UpdateImportedRowCount( - []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) - } + taskImporter, err := NewFileTaskImporter(task, state, batchImportPool, progressReporter) + if err != nil { + utils.ErrExit("Failed to create file task importer: %s", err) } - importFile(state, task, updateProgressFn) - batchImportPool.Wait() // Wait for the file import to finish. - - if importerRole == TARGET_DB_IMPORTER_ROLE { - importDataTableMetrics := createImportDataTableMetrics(task.TableNameTup.ForKey(), - currentProgress, totalProgressAmount, ROW_UPDATE_STATUS_COMPLETED) - controlPlane.UpdateImportedRowCount( - []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) + // totalProgressAmount := getTotalProgressAmount(task) + // progressReporter.ImportFileStarted(task, totalProgressAmount) + // importedProgressAmount := getImportedProgressAmount(task, state) + // progressReporter.AddProgressAmount(task, importedProgressAmount) + + // var currentProgress int64 + // updateProgressFn := func(progressAmount int64) { + // currentProgress += progressAmount + // progressReporter.AddProgressAmount(task, progressAmount) + + // if importerRole == TARGET_DB_IMPORTER_ROLE && totalProgressAmount > currentProgress { + // importDataTableMetrics := createImportDataTableMetrics(task.TableNameTup.ForKey(), + // currentProgress, totalProgressAmount, ROW_UPDATE_STATUS_IN_PROGRESS) + // // The metrics are sent after evry 5 secs in implementation of UpdateImportedRowCount + // controlPlane.UpdateImportedRowCount( + // []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) + // } + // } + + // importFile(state, task, updateProgressFn) + for !taskImporter.AllBatchesSubmitted() { + err := taskImporter.SubmitNextBatch() + if err != nil { + utils.ErrExit("Failed to submit next batch: task:%v err: %s", task, err) + } } - progressReporter.FileImportDone(task) // Remove the progress-bar for the file.\ + batchImportPool.Wait() // Wait for the file import to finish. + taskImporter.PostProcess() + // if importerRole == TARGET_DB_IMPORTER_ROLE { + // importDataTableMetrics := createImportDataTableMetrics(task.TableNameTup.ForKey(), + // currentProgress, totalProgressAmount, ROW_UPDATE_STATUS_COMPLETED) + // controlPlane.UpdateImportedRowCount( + // []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) + // } + + // progressReporter.FileImportDone(task) // Remove the progress-bar for the file.\ } time.Sleep(time.Second * 2) } @@ -950,30 +962,30 @@ func cleanImportState(state *ImportDataState, tasks []*ImportFileTask) { } } -func importFile(state *ImportDataState, task *ImportFileTask, updateProgressFn func(int64)) { +// func importFile(state *ImportDataState, task *ImportFileTask, updateProgressFn func(int64)) { - origDataFile := task.FilePath - importBatchArgsProto := getImportBatchArgsProto(task.TableNameTup, task.FilePath) - log.Infof("Start splitting table %q: data-file: %q", task.TableNameTup, origDataFile) +// origDataFile := task.FilePath +// importBatchArgsProto := getImportBatchArgsProto(task.TableNameTup, task.FilePath) +// log.Infof("Start splitting table %q: data-file: %q", task.TableNameTup, origDataFile) - err := state.PrepareForFileImport(task.FilePath, task.TableNameTup) - if err != nil { - utils.ErrExit("preparing for file import: %s", err) - } +// err := state.PrepareForFileImport(task.FilePath, task.TableNameTup) +// if err != nil { +// utils.ErrExit("preparing for file import: %s", err) +// } - fileBatchProducer, err := NewFileBatchProducer(task, state) - if err != nil { - utils.ErrExit("creating file batch producer: %s", err) - } +// fileBatchProducer, err := NewFileBatchProducer(task, state) +// if err != nil { +// utils.ErrExit("creating file batch producer: %s", err) +// } - for !fileBatchProducer.Done() { - batch, err := fileBatchProducer.NextBatch() - if err != nil { - utils.ErrExit("getting next batch: %s", err) - } - submitBatch(batch, updateProgressFn, importBatchArgsProto) - } -} +// for !fileBatchProducer.Done() { +// batch, err := fileBatchProducer.NextBatch() +// if err != nil { +// utils.ErrExit("getting next batch: %s", err) +// } +// submitBatch(batch, updateProgressFn, importBatchArgsProto) +// } +// } func executePostSnapshotImportSqls() error { sequenceFilePath := filepath.Join(exportDir, "data", "postdata.sql") @@ -987,20 +999,20 @@ func executePostSnapshotImportSqls() error { return nil } -func submitBatch(batch *Batch, updateProgressFn func(int64), importBatchArgsProto *tgtdb.ImportBatchArgs) { - batchImportPool.Go(func() { - // There are `poolSize` number of competing go-routines trying to invoke COPY. - // But the `connPool` will allow only `parallelism` number of connections to be - // used at a time. Thus limiting the number of concurrent COPYs to `parallelism`. - importBatch(batch, importBatchArgsProto) - if reportProgressInBytes { - updateProgressFn(batch.ByteCount) - } else { - updateProgressFn(batch.RecordCount) - } - }) - log.Infof("Queued batch: %s", spew.Sdump(batch)) -} +// func submitBatch(batch *Batch, updateProgressFn func(int64), importBatchArgsProto *tgtdb.ImportBatchArgs) { +// batchImportPool.Go(func() { +// // There are `poolSize` number of competing go-routines trying to invoke COPY. +// // But the `connPool` will allow only `parallelism` number of connections to be +// // used at a time. Thus limiting the number of concurrent COPYs to `parallelism`. +// importBatch(batch, importBatchArgsProto) +// if reportProgressInBytes { +// updateProgressFn(batch.ByteCount) +// } else { +// updateProgressFn(batch.RecordCount) +// } +// }) +// log.Infof("Queued batch: %s", spew.Sdump(batch)) +// } func getIndexName(sqlQuery string, indexName string) (string, error) { // Return the index name itself if it is aleady qualified with schema name From 5dbacbd9ffe41f117c834a6a04b27e8a66513577 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 22 Jan 2025 10:47:03 +0530 Subject: [PATCH 18/87] run tests 2 --- .github/workflows/go.yml | 2 +- .github/workflows/integration-tests.yml | 2 +- .github/workflows/issue-tests.yml | 2 +- .github/workflows/misc-migtests.yml | 2 +- .github/workflows/mysql-migtests.yml | 2 +- .github/workflows/pg-13-migtests.yml | 2 +- .github/workflows/pg-17-migtests.yml | 2 +- .github/workflows/pg-9-migtests.yml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index 2d6a1c8098..e2794811e7 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index 4d71e66fd3..af9410ed55 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/issue-tests.yml b/.github/workflows/issue-tests.yml index 3b72d282fd..40693caeb5 100644 --- a/.github/workflows/issue-tests.yml +++ b/.github/workflows/issue-tests.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/misc-migtests.yml b/.github/workflows/misc-migtests.yml index fdb8fe0e1e..3f7690e4ad 100644 --- a/.github/workflows/misc-migtests.yml +++ b/.github/workflows/misc-migtests.yml @@ -2,7 +2,7 @@ name: "Misc: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/mysql-migtests.yml b/.github/workflows/mysql-migtests.yml index 65e15e2431..4f3b50ef0d 100644 --- a/.github/workflows/mysql-migtests.yml +++ b/.github/workflows/mysql-migtests.yml @@ -2,7 +2,7 @@ name: "MySQL: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-13-migtests.yml b/.github/workflows/pg-13-migtests.yml index 43762a0fa6..2b7b023d49 100644 --- a/.github/workflows/pg-13-migtests.yml +++ b/.github/workflows/pg-13-migtests.yml @@ -2,7 +2,7 @@ name: "PG 13: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-17-migtests.yml b/.github/workflows/pg-17-migtests.yml index 231ec42282..8016d72261 100644 --- a/.github/workflows/pg-17-migtests.yml +++ b/.github/workflows/pg-17-migtests.yml @@ -2,7 +2,7 @@ name: "PG 17: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-9-migtests.yml b/.github/workflows/pg-9-migtests.yml index 44c0e2a81f..7cca6ba363 100644 --- a/.github/workflows/pg-9-migtests.yml +++ b/.github/workflows/pg-9-migtests.yml @@ -2,7 +2,7 @@ name: "PG 9: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] From ae0f5f39da84a08a0aca7ea78d8499181a16c342 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 22 Jan 2025 13:00:08 +0530 Subject: [PATCH 19/87] comments --- yb-voyager/cmd/importDataFileBatchProducer.go | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer.go b/yb-voyager/cmd/importDataFileBatchProducer.go index 4bccbd53f2..037aa11e34 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer.go +++ b/yb-voyager/cmd/importDataFileBatchProducer.go @@ -28,15 +28,17 @@ type FileBatchProducer struct { task *ImportFileTask state *ImportDataState - pendingBatches []*Batch - lastBatchNumber int64 - lastOffset int64 - fileFullySplit bool - completed bool - - dataFile datafile.DataFile - header string - numLinesTaken int64 + pendingBatches []*Batch //pending batches after recovery + lastBatchNumber int64 // batch number of the last batch that was produced + lastOffset int64 // file offset from where the last batch was produced, only used in recovery + fileFullySplit bool // if the file is fully split into batches + completed bool // if all batches have been produced + + dataFile datafile.DataFile + header string + numLinesTaken int64 // number of lines read from the file + // line that was read from file while producing the previous batch + // but not added to the batch because adding it would breach size/row based thresholds. lineFromPreviousBatch string } @@ -150,12 +152,10 @@ func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { p.dataFile.ResetBytesRead(currentBytesRead) p.lineFromPreviousBatch = line - // Start a new batch by calling the initBatchWriter function - // initBatchWriter() return batch, nil } - // Write the record to the new or current batch + // Write the record to the current batch err = batchWriter.WriteRecord(line) if err != nil { return nil, fmt.Errorf("Write to batch %d: %s", batchNum, err) From 634f29c4090749eeb0e6b853b259689d4a241848 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 22 Jan 2025 13:06:30 +0530 Subject: [PATCH 20/87] review comments --- yb-voyager/cmd/importData.go | 5 ----- yb-voyager/cmd/importDataFileBatchProducer.go | 11 +++++++---- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index 09a48e01d8..52d55c83b6 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -1005,11 +1005,6 @@ func importFile(state *ImportDataState, task *ImportFileTask, updateProgressFn f importBatchArgsProto := getImportBatchArgsProto(task.TableNameTup, task.FilePath) log.Infof("Start splitting table %q: data-file: %q", task.TableNameTup, origDataFile) - err := state.PrepareForFileImport(task.FilePath, task.TableNameTup) - if err != nil { - utils.ErrExit("preparing for file import: %s", err) - } - fileBatchProducer, err := NewFileBatchProducer(task, state) if err != nil { utils.ErrExit("creating file batch producer: %s", err) diff --git a/yb-voyager/cmd/importDataFileBatchProducer.go b/yb-voyager/cmd/importDataFileBatchProducer.go index 037aa11e34..8551089cd6 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer.go +++ b/yb-voyager/cmd/importDataFileBatchProducer.go @@ -51,10 +51,7 @@ func NewFileBatchProducer(task *ImportFileTask, state *ImportDataState) (*FileBa if err != nil { return nil, fmt.Errorf("recovering state for table: %q: %s", task.TableNameTup, err) } - var completed bool - if len(pendingBatches) == 0 && fileFullySplit { - completed = true - } + completed := len(pendingBatches) == 0 && fileFullySplit return &FileBatchProducer{ task: task, @@ -106,6 +103,9 @@ func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { if err != nil { return nil, err } + // in the previous batch, a line was read from file but not added to the batch + // because adding it would breach size/row based thresholds. + // Add that line to the current batch. if p.lineFromPreviousBatch != "" { err = batchWriter.WriteRecord(p.lineFromPreviousBatch) if err != nil { @@ -170,6 +170,9 @@ func (p *FileBatchProducer) produceNextBatch() (*Batch, error) { } p.completed = true + // TODO: resetting bytes read to 0 is technically not correct if we are adding a header + // to each batch file. Currently header bytes are only considered in the first batch. + // For the rest of the batches, header bytes are ignored, since we are resetting it to 0. p.dataFile.ResetBytesRead(0) return batch, nil } else if readLineErr != nil { From b2da3ffde76b571d935edd24532f33cb0f5558e7 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 22 Jan 2025 13:50:21 +0530 Subject: [PATCH 21/87] test for when all batches are produced and we resume --- .../cmd/importDataFileBatchProducer_test.go | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index 852dccee60..6d3b528510 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -331,3 +331,57 @@ func TestFileBatchProducerResumable(t *testing.T) { assert.Equal(t, int64(2), batch2.RecordCount) assert.True(t, batchproducer.Done()) } + +func TestFileBatchProducerResumeAfterAllBatchesProduced(t *testing.T) { + // max batch size in rows is 2 + ldataDir, lexportDir, state, err := setupDependenciesForTest(2, 1024) + assert.NoError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + fileContents := `id,val +1, "hello" +2, "world" +3, "foo" +4, "bar"` + _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + assert.NoError(t, err) + + batchproducer, err := NewFileBatchProducer(task, state) + assert.NoError(t, err) + assert.False(t, batchproducer.Done()) + + // generate all batches + batches := []*Batch{} + for !batchproducer.Done() { + batch, err := batchproducer.NextBatch() + assert.NoError(t, err) + assert.NotNil(t, batch) + batches = append(batches, batch) + } + + // simulate a crash and recover + batchproducer, err = NewFileBatchProducer(task, state) + assert.NoError(t, err) + assert.False(t, batchproducer.Done()) + + // state should have recovered two batches + assert.Equal(t, 2, len(batchproducer.pendingBatches)) + + // verify that it picks up from pendingBatches + // instead of procing a new batch. + recoveredBatches := []*Batch{} + for !batchproducer.Done() { + batch, err := batchproducer.NextBatch() + assert.NoError(t, err) + assert.NotNil(t, batch) + recoveredBatches = append(recoveredBatches, batch) + } + assert.Equal(t, len(batches), len(recoveredBatches)) + assert.ElementsMatch(t, batches, recoveredBatches) +} From c65f1126e4c2e44d6035d6e605ad8b7c50831605 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 22 Jan 2025 14:10:37 +0530 Subject: [PATCH 22/87] review comments --- .../cmd/importDataFileBatchProducer_test.go | 67 ++++++++----------- yb-voyager/test/utils/testutils.go | 17 +++++ 2 files changed, 45 insertions(+), 39 deletions(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index 6d3b528510..c33ca04750 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -30,6 +30,7 @@ import ( "github.com/yugabyte/yb-voyager/yb-voyager/src/dbzm" "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" + testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" ) type dummyTDB struct { @@ -41,24 +42,7 @@ func (d *dummyTDB) MaxBatchSizeInBytes() int64 { return d.maxSizeBytes } -func createTempFile(dir string, fileContents string) (string, error) { - // Create a temporary file - file, err := os.CreateTemp(dir, "temp-*.txt") - if err != nil { - return "", err - } - defer file.Close() - - // Write some text to the file - _, err = file.WriteString(fileContents) - if err != nil { - return "", err - } - - return file.Name(), nil -} - -func setupDependenciesForTest(batchSizeRows int64, batchSizeBytes int64) (string, string, *ImportDataState, error) { +func setupExportDirAndImportDependencies(batchSizeRows int64, batchSizeBytes int64) (string, string, *ImportDataState, error) { lexportDir, err := os.MkdirTemp("/tmp", "export-dir-*") if err != nil { return "", "", nil, err @@ -80,7 +64,7 @@ func setupDependenciesForTest(batchSizeRows int64, batchSizeBytes int64) (string return ldataDir, lexportDir, state, nil } -func setupFileForTest(lexportDir string, fileContents string, dir string, tableName string) (string, *ImportFileTask, error) { +func createFileAndTask(lexportDir string, fileContents string, ldataDir string, tableName string) (string, *ImportFileTask, error) { dataFileDescriptor = &datafile.Descriptor{ FileFormat: "csv", Delimiter: ",", @@ -90,7 +74,7 @@ func setupFileForTest(lexportDir string, fileContents string, dir string, tableN EscapeChar: '\\', NullString: "NULL", } - tempFile, err := createTempFile(dir, fileContents) + tempFile, err := testutils.CreateTempFile(ldataDir, fileContents) if err != nil { return "", nil, err } @@ -107,7 +91,7 @@ func setupFileForTest(lexportDir string, fileContents string, dir string, tableN } func TestBasicFileBatchProducer(t *testing.T) { - ldataDir, lexportDir, state, err := setupDependenciesForTest(2, 1024) + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) assert.NoError(t, err) if ldataDir != "" { @@ -119,7 +103,7 @@ func TestBasicFileBatchProducer(t *testing.T) { fileContents := `id,val 1, "hello"` - _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -136,7 +120,7 @@ func TestBasicFileBatchProducer(t *testing.T) { func TestFileBatchProducerBasedOnRowsThreshold(t *testing.T) { // max batch size in rows is 2 - ldataDir, lexportDir, state, err := setupDependenciesForTest(2, 1024) + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) assert.NoError(t, err) if ldataDir != "" { @@ -151,7 +135,7 @@ func TestFileBatchProducerBasedOnRowsThreshold(t *testing.T) { 2, "world" 3, "foo" 4, "bar"` - _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -169,22 +153,24 @@ func TestFileBatchProducerBasedOnRowsThreshold(t *testing.T) { // 2 batches should be produced assert.Equal(t, 2, len(batches)) - // each of length 2 + + batch1ExpectedContents := "id,val\n1, \"hello\"\n2, \"world\"" assert.Equal(t, int64(2), batches[0].RecordCount) batchContents, err := os.ReadFile(batches[0].GetFilePath()) assert.NoError(t, err) - assert.Equal(t, "id,val\n1, \"hello\"\n2, \"world\"", string(batchContents)) + assert.Equal(t, batch1ExpectedContents, string(batchContents)) + batch2ExpectedContents := "id,val\n3, \"foo\"\n4, \"bar\"" assert.Equal(t, int64(2), batches[1].RecordCount) batchContents, err = os.ReadFile(batches[1].GetFilePath()) assert.NoError(t, err) - assert.Equal(t, "id,val\n3, \"foo\"\n4, \"bar\"", string(batchContents)) + assert.Equal(t, batch2ExpectedContents, string(batchContents)) } func TestFileBatchProducerBasedOnSizeThreshold(t *testing.T) { // max batch size in size is 25 bytes maxBatchSizeBytes := int64(25) - ldataDir, lexportDir, state, err := setupDependenciesForTest(1000, maxBatchSizeBytes) + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(1000, maxBatchSizeBytes) assert.NoError(t, err) if ldataDir != "" { @@ -200,7 +186,7 @@ func TestFileBatchProducerBasedOnSizeThreshold(t *testing.T) { 2, "ghijk" 3, "mnopq" 4, "stuvw"` - _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -219,29 +205,32 @@ func TestFileBatchProducerBasedOnSizeThreshold(t *testing.T) { // 3 batches should be produced // while calculating for the first batch, the header is also considered assert.Equal(t, 3, len(batches)) - // each of length 2 + + batch1ExpectedContents := "id,val\n1, \"abcde\"" assert.Equal(t, int64(1), batches[0].RecordCount) assert.LessOrEqual(t, batches[0].ByteCount, maxBatchSizeBytes) batchContents, err := os.ReadFile(batches[0].GetFilePath()) assert.NoError(t, err) - assert.Equal(t, "id,val\n1, \"abcde\"", string(batchContents)) + assert.Equal(t, batch1ExpectedContents, string(batchContents)) + batch2ExpectedContents := "id,val\n2, \"ghijk\"\n3, \"mnopq\"" assert.Equal(t, int64(2), batches[1].RecordCount) assert.LessOrEqual(t, batches[1].ByteCount, maxBatchSizeBytes) batchContents, err = os.ReadFile(batches[1].GetFilePath()) assert.NoError(t, err) - assert.Equal(t, "id,val\n2, \"ghijk\"\n3, \"mnopq\"", string(batchContents)) + assert.Equal(t, batch2ExpectedContents, string(batchContents)) + batch3ExpectedContents := "id,val\n4, \"stuvw\"" assert.Equal(t, int64(1), batches[2].RecordCount) assert.LessOrEqual(t, batches[2].ByteCount, maxBatchSizeBytes) batchContents, err = os.ReadFile(batches[2].GetFilePath()) assert.NoError(t, err) - assert.Equal(t, "id,val\n4, \"stuvw\"", string(batchContents)) + assert.Equal(t, batch3ExpectedContents, string(batchContents)) } func TestFileBatchProducerThrowsErrorWhenSingleRowGreaterThanMaxBatchSize(t *testing.T) { // max batch size in size is 25 bytes - ldataDir, lexportDir, state, err := setupDependenciesForTest(1000, 25) + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(1000, 25) assert.NoError(t, err) if ldataDir != "" { @@ -257,7 +246,7 @@ func TestFileBatchProducerThrowsErrorWhenSingleRowGreaterThanMaxBatchSize(t *tes 2, "ghijk" 3, "mnopq1234567899876543" 4, "stuvw"` - _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -278,7 +267,7 @@ func TestFileBatchProducerThrowsErrorWhenSingleRowGreaterThanMaxBatchSize(t *tes func TestFileBatchProducerResumable(t *testing.T) { // max batch size in rows is 2 - ldataDir, lexportDir, state, err := setupDependenciesForTest(2, 1024) + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) assert.NoError(t, err) if ldataDir != "" { @@ -293,7 +282,7 @@ func TestFileBatchProducerResumable(t *testing.T) { 2, "world" 3, "foo" 4, "bar"` - _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -334,7 +323,7 @@ func TestFileBatchProducerResumable(t *testing.T) { func TestFileBatchProducerResumeAfterAllBatchesProduced(t *testing.T) { // max batch size in rows is 2 - ldataDir, lexportDir, state, err := setupDependenciesForTest(2, 1024) + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) assert.NoError(t, err) if ldataDir != "" { @@ -349,7 +338,7 @@ func TestFileBatchProducerResumeAfterAllBatchesProduced(t *testing.T) { 2, "world" 3, "foo" 4, "bar"` - _, task, err := setupFileForTest(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) diff --git a/yb-voyager/test/utils/testutils.go b/yb-voyager/test/utils/testutils.go index 942e00fd67..f8a3418ab0 100644 --- a/yb-voyager/test/utils/testutils.go +++ b/yb-voyager/test/utils/testutils.go @@ -406,3 +406,20 @@ func FatalIfError(t *testing.T, err error) { t.Fatalf("error: %v", err) } } + +func CreateTempFile(dir string, fileContents string) (string, error) { + // Create a temporary file + file, err := os.CreateTemp(dir, "temp-*.txt") + if err != nil { + return "", err + } + defer file.Close() + + // Write some text to the file + _, err = file.WriteString(fileContents) + if err != nil { + return "", err + } + + return file.Name(), nil +} From f9babc5025365bb9db5f93755177fd7abe98fd2f Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 10:32:04 +0530 Subject: [PATCH 23/87] Revert "run tests 2" This reverts commit 5dbacbd9ffe41f117c834a6a04b27e8a66513577. --- .github/workflows/go.yml | 2 +- .github/workflows/integration-tests.yml | 2 +- .github/workflows/issue-tests.yml | 2 +- .github/workflows/misc-migtests.yml | 2 +- .github/workflows/mysql-migtests.yml | 2 +- .github/workflows/pg-13-migtests.yml | 2 +- .github/workflows/pg-17-migtests.yml | 2 +- .github/workflows/pg-9-migtests.yml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index e2794811e7..2d6a1c8098 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index af9410ed55..4d71e66fd3 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/issue-tests.yml b/.github/workflows/issue-tests.yml index 40693caeb5..3b72d282fd 100644 --- a/.github/workflows/issue-tests.yml +++ b/.github/workflows/issue-tests.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/misc-migtests.yml b/.github/workflows/misc-migtests.yml index 37dd3c0492..69fbe4dffe 100644 --- a/.github/workflows/misc-migtests.yml +++ b/.github/workflows/misc-migtests.yml @@ -2,7 +2,7 @@ name: "Misc: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/mysql-migtests.yml b/.github/workflows/mysql-migtests.yml index 04ce8bf52f..230a347ee2 100644 --- a/.github/workflows/mysql-migtests.yml +++ b/.github/workflows/mysql-migtests.yml @@ -2,7 +2,7 @@ name: "MySQL: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-13-migtests.yml b/.github/workflows/pg-13-migtests.yml index b761072bf4..dd29300a2b 100644 --- a/.github/workflows/pg-13-migtests.yml +++ b/.github/workflows/pg-13-migtests.yml @@ -2,7 +2,7 @@ name: "PG 13: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-17-migtests.yml b/.github/workflows/pg-17-migtests.yml index 695a873dc1..0b4e702c42 100644 --- a/.github/workflows/pg-17-migtests.yml +++ b/.github/workflows/pg-17-migtests.yml @@ -2,7 +2,7 @@ name: "PG 17: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-9-migtests.yml b/.github/workflows/pg-9-migtests.yml index 1647827243..2c32ced464 100644 --- a/.github/workflows/pg-9-migtests.yml +++ b/.github/workflows/pg-9-migtests.yml @@ -2,7 +2,7 @@ name: "PG 9: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] From 2f1e07043efc41b0df45e6153d8a8387842f73c9 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 10:32:17 +0530 Subject: [PATCH 24/87] Revert "run tests" This reverts commit 1e4b1c8dcb3a94d353253aebd02b4bbc499af197. --- .github/workflows/go.yml | 2 +- .github/workflows/integration-tests.yml | 2 +- .github/workflows/issue-tests.yml | 2 +- .github/workflows/misc-migtests.yml | 2 +- .github/workflows/mysql-migtests.yml | 2 +- .github/workflows/pg-13-migtests.yml | 2 +- .github/workflows/pg-17-migtests.yml | 2 +- .github/workflows/pg-9-migtests.yml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index 2d6a1c8098..88c20428d5 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: [main] jobs: diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index 4d71e66fd3..d94a594cb1 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: [main] env: ORACLE_INSTANT_CLIENT_VERSION: "21.5.0.0.0-1" diff --git a/.github/workflows/issue-tests.yml b/.github/workflows/issue-tests.yml index 3b72d282fd..550eef3c8a 100644 --- a/.github/workflows/issue-tests.yml +++ b/.github/workflows/issue-tests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: [main] jobs: diff --git a/.github/workflows/misc-migtests.yml b/.github/workflows/misc-migtests.yml index 69fbe4dffe..053de031e4 100644 --- a/.github/workflows/misc-migtests.yml +++ b/.github/workflows/misc-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-misc-migration-tests: diff --git a/.github/workflows/mysql-migtests.yml b/.github/workflows/mysql-migtests.yml index 230a347ee2..2242861873 100644 --- a/.github/workflows/mysql-migtests.yml +++ b/.github/workflows/mysql-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-mysql-migration-tests: diff --git a/.github/workflows/pg-13-migtests.yml b/.github/workflows/pg-13-migtests.yml index dd29300a2b..bcb78f4338 100644 --- a/.github/workflows/pg-13-migtests.yml +++ b/.github/workflows/pg-13-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-pg-13-migration-tests: diff --git a/.github/workflows/pg-17-migtests.yml b/.github/workflows/pg-17-migtests.yml index 0b4e702c42..771d20f8d1 100644 --- a/.github/workflows/pg-17-migtests.yml +++ b/.github/workflows/pg-17-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-pg-17-migration-tests: diff --git a/.github/workflows/pg-9-migtests.yml b/.github/workflows/pg-9-migtests.yml index 2c32ced464..7fb10ef6e2 100644 --- a/.github/workflows/pg-9-migtests.yml +++ b/.github/workflows/pg-9-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-pg-9-migration-tests: From 2e8d241f7a7896a49a9ef1bdb3d9298fe47cd129 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 11:03:33 +0530 Subject: [PATCH 25/87] basic test --- .../cmd/importDataFileTaskImporter_test.go | 104 ++++++++++++++++++ 1 file changed, 104 insertions(+) create mode 100644 yb-voyager/cmd/importDataFileTaskImporter_test.go diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go new file mode 100644 index 0000000000..b9b8451e7e --- /dev/null +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -0,0 +1,104 @@ +/* +Copyright (c) YugabyteDB, Inc. + +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 cmd + +import ( + "context" + "fmt" + "os" + "testing" + + "github.com/sourcegraph/conc/pool" + "github.com/stretchr/testify/assert" + "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" + testcontainers "github.com/yugabyte/yb-voyager/yb-voyager/test/containers" + testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" +) + +type TestDB struct { + testcontainers.TestContainer + tgtdb.TargetDB +} + +func TestBasicTaskImport(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + // yb + yugabytedbContainer := testcontainers.NewTestContainer("yugabytedb", nil) + err = yugabytedbContainer.Start(context.Background()) + testutils.FatalIfError(t, err) + host, port, err := yugabytedbContainer.GetHostPort() + testutils.FatalIfError(t, err) + testYugabyteDBTarget := &TestDB{ + TestContainer: yugabytedbContainer, + TargetDB: tgtdb.NewTargetDB(&tgtdb.TargetConf{ + TargetDBType: "yugabytedb", + DBVersion: yugabytedbContainer.GetConfig().DBVersion, + User: yugabytedbContainer.GetConfig().User, + Password: yugabytedbContainer.GetConfig().Password, + Schema: yugabytedbContainer.GetConfig().Schema, + DBName: yugabytedbContainer.GetConfig().DBName, + Host: host, + Port: port, + }), + } + testYugabyteDBTarget.TestContainer.ExecuteSqls( + `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, + ) + + err = testYugabyteDBTarget.Init() + testutils.FatalIfError(t, err) + defer testYugabyteDBTarget.Finalize() + tdb = testYugabyteDBTarget.TargetDB + err = tdb.CreateVoyagerSchema() + testutils.FatalIfError(t, err) + err = tdb.InitConnPool() + testutils.FatalIfError(t, err) + + TableNameToSchema = utils.NewStructMap[sqlname.NameTuple, map[string]map[string]string]() + + // file import + fileContents := `id,val +1, "hello" +2, "world"` + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + testutils.FatalIfError(t, err) + + progressReporter := NewImportDataProgressReporter(true) + workerPool := pool.New().WithMaxGoroutines(2) + taskImporter, err := NewFileTaskImporter(task, state, workerPool, progressReporter) + + for !taskImporter.AllBatchesSubmitted() { + err := taskImporter.SubmitNextBatch() + assert.NoError(t, err) + } + + workerPool.Wait() + var rowCount int64 + err = tdb.QueryRow("SELECT count(*) FROM test_table").Scan(&rowCount) + assert.NoError(t, err) + assert.Equal(t, int64(2), rowCount) +} From 92a8254167b4125eae844513634ab6d613e677ef Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 16:34:11 +0530 Subject: [PATCH 26/87] minor refactor --- .../cmd/importDataFileBatchProducer_test.go | 2 ++ .../cmd/importDataFileTaskImporter_test.go | 35 +++++++++---------- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index c33ca04750..f15b9579b8 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -29,6 +29,7 @@ import ( "github.com/yugabyte/yb-voyager/yb-voyager/src/datastore" "github.com/yugabyte/yb-voyager/yb-voyager/src/dbzm" "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" ) @@ -61,6 +62,7 @@ func setupExportDirAndImportDependencies(batchSizeRows int64, batchSizeBytes int batchSizeInNumRows = batchSizeRows state := NewImportDataState(lexportDir) + TableNameToSchema = utils.NewStructMap[sqlname.NameTuple, map[string]map[string]string]() return ldataDir, lexportDir, state, nil } diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go index b9b8451e7e..d7cd55ce99 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter_test.go +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -24,8 +24,6 @@ import ( "github.com/sourcegraph/conc/pool" "github.com/stretchr/testify/assert" "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" - "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" - "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" testcontainers "github.com/yugabyte/yb-voyager/yb-voyager/test/containers" testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" ) @@ -35,20 +33,11 @@ type TestDB struct { tgtdb.TargetDB } -func TestBasicTaskImport(t *testing.T) { - ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) - testutils.FatalIfError(t, err) - - if ldataDir != "" { - defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) - } - if lexportDir != "" { - defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) - } +var testYugabyteDBTarget *TestDB - // yb +func setupYugabyteTestDb(t *testing.T) { yugabytedbContainer := testcontainers.NewTestContainer("yugabytedb", nil) - err = yugabytedbContainer.Start(context.Background()) + err := yugabytedbContainer.Start(context.Background()) testutils.FatalIfError(t, err) host, port, err := yugabytedbContainer.GetHostPort() testutils.FatalIfError(t, err) @@ -69,16 +58,26 @@ func TestBasicTaskImport(t *testing.T) { `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, ) - err = testYugabyteDBTarget.Init() - testutils.FatalIfError(t, err) - defer testYugabyteDBTarget.Finalize() tdb = testYugabyteDBTarget.TargetDB + err = tdb.Init() + testutils.FatalIfError(t, err) err = tdb.CreateVoyagerSchema() testutils.FatalIfError(t, err) err = tdb.InitConnPool() testutils.FatalIfError(t, err) +} + +func TestBasicTaskImport(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) - TableNameToSchema = utils.NewStructMap[sqlname.NameTuple, map[string]map[string]string]() + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + setupYugabyteTestDb(t) // file import fileContents := `id,val From 66d2c157ebec8a645a3793c84bdc1b2b7da98313 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 16:41:49 +0530 Subject: [PATCH 27/87] small refactor of test utils --- .../cmd/importDataFileBatchProducer_test.go | 66 --------- .../cmd/importDataFileTaskImporter_test.go | 42 ------ yb-voyager/cmd/importDataTestUtils.go | 129 ++++++++++++++++++ 3 files changed, 129 insertions(+), 108 deletions(-) create mode 100644 yb-voyager/cmd/importDataTestUtils.go diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index f15b9579b8..4b3f171105 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -24,74 +24,8 @@ import ( "github.com/google/go-cmp/cmp" "github.com/stretchr/testify/assert" - "github.com/yugabyte/yb-voyager/yb-voyager/src/constants" - "github.com/yugabyte/yb-voyager/yb-voyager/src/datafile" - "github.com/yugabyte/yb-voyager/yb-voyager/src/datastore" - "github.com/yugabyte/yb-voyager/yb-voyager/src/dbzm" - "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" - "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" - "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" - testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" ) -type dummyTDB struct { - maxSizeBytes int64 - tgtdb.TargetYugabyteDB -} - -func (d *dummyTDB) MaxBatchSizeInBytes() int64 { - return d.maxSizeBytes -} - -func setupExportDirAndImportDependencies(batchSizeRows int64, batchSizeBytes int64) (string, string, *ImportDataState, error) { - lexportDir, err := os.MkdirTemp("/tmp", "export-dir-*") - if err != nil { - return "", "", nil, err - } - - ldataDir, err := os.MkdirTemp("/tmp", "data-dir-*") - if err != nil { - return "", "", nil, err - } - - CreateMigrationProjectIfNotExists(constants.POSTGRESQL, lexportDir) - tdb = &dummyTDB{maxSizeBytes: batchSizeBytes} - valueConverter = &dbzm.NoOpValueConverter{} - dataStore = datastore.NewDataStore(ldataDir) - - batchSizeInNumRows = batchSizeRows - - state := NewImportDataState(lexportDir) - TableNameToSchema = utils.NewStructMap[sqlname.NameTuple, map[string]map[string]string]() - return ldataDir, lexportDir, state, nil -} - -func createFileAndTask(lexportDir string, fileContents string, ldataDir string, tableName string) (string, *ImportFileTask, error) { - dataFileDescriptor = &datafile.Descriptor{ - FileFormat: "csv", - Delimiter: ",", - HasHeader: true, - ExportDir: lexportDir, - QuoteChar: '"', - EscapeChar: '\\', - NullString: "NULL", - } - tempFile, err := testutils.CreateTempFile(ldataDir, fileContents) - if err != nil { - return "", nil, err - } - - sourceName := sqlname.NewObjectName(constants.POSTGRESQL, "public", "public", tableName) - tableNameTup := sqlname.NameTuple{SourceName: sourceName, CurrentName: sourceName} - task := &ImportFileTask{ - ID: 1, - FilePath: tempFile, - TableNameTup: tableNameTup, - RowCount: 1, - } - return tempFile, task, nil -} - func TestBasicFileBatchProducer(t *testing.T) { ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) assert.NoError(t, err) diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go index d7cd55ce99..61ab38a5ad 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter_test.go +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -16,57 +16,15 @@ limitations under the License. package cmd import ( - "context" "fmt" "os" "testing" "github.com/sourcegraph/conc/pool" "github.com/stretchr/testify/assert" - "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" - testcontainers "github.com/yugabyte/yb-voyager/yb-voyager/test/containers" testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" ) -type TestDB struct { - testcontainers.TestContainer - tgtdb.TargetDB -} - -var testYugabyteDBTarget *TestDB - -func setupYugabyteTestDb(t *testing.T) { - yugabytedbContainer := testcontainers.NewTestContainer("yugabytedb", nil) - err := yugabytedbContainer.Start(context.Background()) - testutils.FatalIfError(t, err) - host, port, err := yugabytedbContainer.GetHostPort() - testutils.FatalIfError(t, err) - testYugabyteDBTarget := &TestDB{ - TestContainer: yugabytedbContainer, - TargetDB: tgtdb.NewTargetDB(&tgtdb.TargetConf{ - TargetDBType: "yugabytedb", - DBVersion: yugabytedbContainer.GetConfig().DBVersion, - User: yugabytedbContainer.GetConfig().User, - Password: yugabytedbContainer.GetConfig().Password, - Schema: yugabytedbContainer.GetConfig().Schema, - DBName: yugabytedbContainer.GetConfig().DBName, - Host: host, - Port: port, - }), - } - testYugabyteDBTarget.TestContainer.ExecuteSqls( - `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, - ) - - tdb = testYugabyteDBTarget.TargetDB - err = tdb.Init() - testutils.FatalIfError(t, err) - err = tdb.CreateVoyagerSchema() - testutils.FatalIfError(t, err) - err = tdb.InitConnPool() - testutils.FatalIfError(t, err) -} - func TestBasicTaskImport(t *testing.T) { ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) testutils.FatalIfError(t, err) diff --git a/yb-voyager/cmd/importDataTestUtils.go b/yb-voyager/cmd/importDataTestUtils.go new file mode 100644 index 0000000000..770769de10 --- /dev/null +++ b/yb-voyager/cmd/importDataTestUtils.go @@ -0,0 +1,129 @@ +/* +Copyright (c) YugabyteDB, Inc. + +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 cmd + +import ( + "context" + "os" + "testing" + + "github.com/yugabyte/yb-voyager/yb-voyager/src/constants" + "github.com/yugabyte/yb-voyager/yb-voyager/src/datafile" + "github.com/yugabyte/yb-voyager/yb-voyager/src/datastore" + "github.com/yugabyte/yb-voyager/yb-voyager/src/dbzm" + "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" + testcontainers "github.com/yugabyte/yb-voyager/yb-voyager/test/containers" + testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" +) + +type dummyTDB struct { + maxSizeBytes int64 + tgtdb.TargetYugabyteDB +} + +func (d *dummyTDB) MaxBatchSizeInBytes() int64 { + return d.maxSizeBytes +} + +type TestDB struct { + testcontainers.TestContainer + tgtdb.TargetDB +} + +var testYugabyteDBTarget *TestDB + +func setupYugabyteTestDb(t *testing.T) { + yugabytedbContainer := testcontainers.NewTestContainer("yugabytedb", nil) + err := yugabytedbContainer.Start(context.Background()) + testutils.FatalIfError(t, err) + host, port, err := yugabytedbContainer.GetHostPort() + testutils.FatalIfError(t, err) + testYugabyteDBTarget := &TestDB{ + TestContainer: yugabytedbContainer, + TargetDB: tgtdb.NewTargetDB(&tgtdb.TargetConf{ + TargetDBType: "yugabytedb", + DBVersion: yugabytedbContainer.GetConfig().DBVersion, + User: yugabytedbContainer.GetConfig().User, + Password: yugabytedbContainer.GetConfig().Password, + Schema: yugabytedbContainer.GetConfig().Schema, + DBName: yugabytedbContainer.GetConfig().DBName, + Host: host, + Port: port, + }), + } + testYugabyteDBTarget.TestContainer.ExecuteSqls( + `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, + ) + + tdb = testYugabyteDBTarget.TargetDB + err = tdb.Init() + testutils.FatalIfError(t, err) + err = tdb.CreateVoyagerSchema() + testutils.FatalIfError(t, err) + err = tdb.InitConnPool() + testutils.FatalIfError(t, err) +} + +func setupExportDirAndImportDependencies(batchSizeRows int64, batchSizeBytes int64) (string, string, *ImportDataState, error) { + lexportDir, err := os.MkdirTemp("/tmp", "export-dir-*") + if err != nil { + return "", "", nil, err + } + + ldataDir, err := os.MkdirTemp("/tmp", "data-dir-*") + if err != nil { + return "", "", nil, err + } + + CreateMigrationProjectIfNotExists(constants.POSTGRESQL, lexportDir) + tdb = &dummyTDB{maxSizeBytes: batchSizeBytes} + valueConverter = &dbzm.NoOpValueConverter{} + dataStore = datastore.NewDataStore(ldataDir) + + batchSizeInNumRows = batchSizeRows + + state := NewImportDataState(lexportDir) + TableNameToSchema = utils.NewStructMap[sqlname.NameTuple, map[string]map[string]string]() + return ldataDir, lexportDir, state, nil +} + +func createFileAndTask(lexportDir string, fileContents string, ldataDir string, tableName string) (string, *ImportFileTask, error) { + dataFileDescriptor = &datafile.Descriptor{ + FileFormat: "csv", + Delimiter: ",", + HasHeader: true, + ExportDir: lexportDir, + QuoteChar: '"', + EscapeChar: '\\', + NullString: "NULL", + } + tempFile, err := testutils.CreateTempFile(ldataDir, fileContents) + if err != nil { + return "", nil, err + } + + sourceName := sqlname.NewObjectName(constants.POSTGRESQL, "public", "public", tableName) + tableNameTup := sqlname.NameTuple{SourceName: sourceName, CurrentName: sourceName} + task := &ImportFileTask{ + ID: 1, + FilePath: tempFile, + TableNameTup: tableNameTup, + RowCount: 1, + } + return tempFile, task, nil +} From a5b0527a97a9edd102b2d5f89f21c5d83f09a959 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 16:52:27 +0530 Subject: [PATCH 28/87] resumability test --- .../cmd/importDataFileTaskImporter_test.go | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go index 61ab38a5ad..33f1ce0591 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter_test.go +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -59,3 +59,60 @@ func TestBasicTaskImport(t *testing.T) { assert.NoError(t, err) assert.Equal(t, int64(2), rowCount) } + +func TestTaskImportResumable(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + setupYugabyteTestDb(t) + + // file import + fileContents := `id,val +1, "hello" +2, "world" +3, "foo" +4, "bar"` + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + testutils.FatalIfError(t, err) + + progressReporter := NewImportDataProgressReporter(true) + workerPool := pool.New().WithMaxGoroutines(2) + taskImporter, err := NewFileTaskImporter(task, state, workerPool, progressReporter) + testutils.FatalIfError(t, err) + // for !taskImporter.AllBatchesSubmitted() { + // err := taskImporter.SubmitNextBatch() + // assert.NoError(t, err) + // } + + // submit 1 batch + err = taskImporter.SubmitNextBatch() + assert.NoError(t, err) + + // simulate restart + workerPool.Wait() + // check that the first batch was imported + var rowCount int64 + err = tdb.QueryRow("SELECT count(*) FROM test_table").Scan(&rowCount) + assert.NoError(t, err) + assert.Equal(t, int64(2), rowCount) + + progressReporter = NewImportDataProgressReporter(true) + workerPool = pool.New().WithMaxGoroutines(2) + taskImporter, err = NewFileTaskImporter(task, state, workerPool, progressReporter) + + // submit second batch, not first batch again as it was already imported + err = taskImporter.SubmitNextBatch() + assert.NoError(t, err) + + assert.Equal(t, true, taskImporter.AllBatchesSubmitted()) + workerPool.Wait() + err = tdb.QueryRow("SELECT count(*) FROM test_table").Scan(&rowCount) + assert.NoError(t, err) + assert.Equal(t, int64(4), rowCount) +} From ca46d32cb6709b9fbb9e32c463c2b79d40c73303 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 17:04:44 +0530 Subject: [PATCH 29/87] fix cleanup --- .../cmd/importDataFileTaskImporter_test.go | 68 +++++++++++++++++-- yb-voyager/cmd/importDataTestUtils.go | 5 +- 2 files changed, 65 insertions(+), 8 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go index 33f1ce0591..464c0fedc2 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter_test.go +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -36,6 +36,54 @@ func TestBasicTaskImport(t *testing.T) { defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) } setupYugabyteTestDb(t) + defer testYugabyteDBTarget.Finalize() + // defer testYugabyteDBTarget.TestContainer.Terminate(context.Background()) + testYugabyteDBTarget.TestContainer.ExecuteSqls( + `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, + ) + defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table;`) + + // file import + fileContents := `id,val +1, "hello" +2, "world"` + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + testutils.FatalIfError(t, err) + + progressReporter := NewImportDataProgressReporter(true) + workerPool := pool.New().WithMaxGoroutines(2) + taskImporter, err := NewFileTaskImporter(task, state, workerPool, progressReporter) + testutils.FatalIfError(t, err) + + for !taskImporter.AllBatchesSubmitted() { + err := taskImporter.SubmitNextBatch() + assert.NoError(t, err) + } + + workerPool.Wait() + var rowCount int64 + err = tdb.QueryRow("SELECT count(*) FROM test_table").Scan(&rowCount) + assert.NoError(t, err) + assert.Equal(t, int64(2), rowCount) +} + +func TestImportAllBatchesAndResume(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + setupYugabyteTestDb(t) + defer testYugabyteDBTarget.Finalize() + // defer testYugabyteDBTarget.TestContainer.Terminate(context.Background()) + testYugabyteDBTarget.TestContainer.ExecuteSqls( + `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, + ) + defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table;`) // file import fileContents := `id,val @@ -58,6 +106,15 @@ func TestBasicTaskImport(t *testing.T) { err = tdb.QueryRow("SELECT count(*) FROM test_table").Scan(&rowCount) assert.NoError(t, err) assert.Equal(t, int64(2), rowCount) + + // simulate restart + progressReporter = NewImportDataProgressReporter(true) + workerPool = pool.New().WithMaxGoroutines(2) + taskImporter, err = NewFileTaskImporter(task, state, workerPool, progressReporter) + testutils.FatalIfError(t, err) + + assert.Equal(t, true, taskImporter.AllBatchesSubmitted()) + // assert.Equal(t, true, taskImporter.AllBatchesImported()) } func TestTaskImportResumable(t *testing.T) { @@ -71,6 +128,12 @@ func TestTaskImportResumable(t *testing.T) { defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) } setupYugabyteTestDb(t) + defer testYugabyteDBTarget.Finalize() + // defer testYugabyteDBTarget.TestContainer.Terminate(context.Background()) + testYugabyteDBTarget.TestContainer.ExecuteSqls( + `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, + ) + defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table;`) // file import fileContents := `id,val @@ -85,10 +148,6 @@ func TestTaskImportResumable(t *testing.T) { workerPool := pool.New().WithMaxGoroutines(2) taskImporter, err := NewFileTaskImporter(task, state, workerPool, progressReporter) testutils.FatalIfError(t, err) - // for !taskImporter.AllBatchesSubmitted() { - // err := taskImporter.SubmitNextBatch() - // assert.NoError(t, err) - // } // submit 1 batch err = taskImporter.SubmitNextBatch() @@ -105,6 +164,7 @@ func TestTaskImportResumable(t *testing.T) { progressReporter = NewImportDataProgressReporter(true) workerPool = pool.New().WithMaxGoroutines(2) taskImporter, err = NewFileTaskImporter(task, state, workerPool, progressReporter) + testutils.FatalIfError(t, err) // submit second batch, not first batch again as it was already imported err = taskImporter.SubmitNextBatch() diff --git a/yb-voyager/cmd/importDataTestUtils.go b/yb-voyager/cmd/importDataTestUtils.go index de2c84829c..6c6ef4cd56 100644 --- a/yb-voyager/cmd/importDataTestUtils.go +++ b/yb-voyager/cmd/importDataTestUtils.go @@ -53,7 +53,7 @@ func setupYugabyteTestDb(t *testing.T) { testutils.FatalIfError(t, err) host, port, err := yugabytedbContainer.GetHostPort() testutils.FatalIfError(t, err) - testYugabyteDBTarget := &TestDB{ + testYugabyteDBTarget = &TestDB{ TestContainer: yugabytedbContainer, TargetDB: tgtdb.NewTargetDB(&tgtdb.TargetConf{ TargetDBType: "yugabytedb", @@ -66,9 +66,6 @@ func setupYugabyteTestDb(t *testing.T) { Port: port, }), } - testYugabyteDBTarget.TestContainer.ExecuteSqls( - `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, - ) tdb = testYugabyteDBTarget.TargetDB err = tdb.Init() From 530e31c920f207cc4c72517c7d33c67857c7ca87 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 17:05:37 +0530 Subject: [PATCH 30/87] integration tag --- yb-voyager/cmd/importDataFileTaskImporter_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go index 464c0fedc2..673416235e 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter_test.go +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -1,3 +1,5 @@ +//go:build integration + /* Copyright (c) YugabyteDB, Inc. From 8871faca261e302b1ecce5025939105c42410756 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 17:17:18 +0530 Subject: [PATCH 31/87] different table names to avoid read restart errors --- .../cmd/importDataFileTaskImporter_test.go | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go index 673416235e..add623f50f 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter_test.go +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -41,15 +41,15 @@ func TestBasicTaskImport(t *testing.T) { defer testYugabyteDBTarget.Finalize() // defer testYugabyteDBTarget.TestContainer.Terminate(context.Background()) testYugabyteDBTarget.TestContainer.ExecuteSqls( - `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, + `CREATE TABLE test_table_basic (id INT PRIMARY KEY, val TEXT);`, ) - defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table;`) + defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table_basic;`) // file import fileContents := `id,val 1, "hello" 2, "world"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_basic") testutils.FatalIfError(t, err) progressReporter := NewImportDataProgressReporter(true) @@ -64,7 +64,7 @@ func TestBasicTaskImport(t *testing.T) { workerPool.Wait() var rowCount int64 - err = tdb.QueryRow("SELECT count(*) FROM test_table").Scan(&rowCount) + err = tdb.QueryRow("SELECT count(*) FROM test_table_basic").Scan(&rowCount) assert.NoError(t, err) assert.Equal(t, int64(2), rowCount) } @@ -83,15 +83,15 @@ func TestImportAllBatchesAndResume(t *testing.T) { defer testYugabyteDBTarget.Finalize() // defer testYugabyteDBTarget.TestContainer.Terminate(context.Background()) testYugabyteDBTarget.TestContainer.ExecuteSqls( - `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, + `CREATE TABLE test_table_all (id INT PRIMARY KEY, val TEXT);`, ) - defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table;`) + defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table_all;`) // file import fileContents := `id,val 1, "hello" 2, "world"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_all") testutils.FatalIfError(t, err) progressReporter := NewImportDataProgressReporter(true) @@ -105,7 +105,7 @@ func TestImportAllBatchesAndResume(t *testing.T) { workerPool.Wait() var rowCount int64 - err = tdb.QueryRow("SELECT count(*) FROM test_table").Scan(&rowCount) + err = tdb.QueryRow("SELECT count(*) FROM test_table_all").Scan(&rowCount) assert.NoError(t, err) assert.Equal(t, int64(2), rowCount) @@ -133,9 +133,9 @@ func TestTaskImportResumable(t *testing.T) { defer testYugabyteDBTarget.Finalize() // defer testYugabyteDBTarget.TestContainer.Terminate(context.Background()) testYugabyteDBTarget.TestContainer.ExecuteSqls( - `CREATE TABLE test_table (id INT PRIMARY KEY, val TEXT);`, + `CREATE TABLE test_table_resume (id INT PRIMARY KEY, val TEXT);`, ) - defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table;`) + defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table_resume;`) // file import fileContents := `id,val @@ -143,7 +143,7 @@ func TestTaskImportResumable(t *testing.T) { 2, "world" 3, "foo" 4, "bar"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_resume") testutils.FatalIfError(t, err) progressReporter := NewImportDataProgressReporter(true) @@ -159,7 +159,7 @@ func TestTaskImportResumable(t *testing.T) { workerPool.Wait() // check that the first batch was imported var rowCount int64 - err = tdb.QueryRow("SELECT count(*) FROM test_table").Scan(&rowCount) + err = tdb.QueryRow("SELECT count(*) FROM test_table_resume").Scan(&rowCount) assert.NoError(t, err) assert.Equal(t, int64(2), rowCount) @@ -174,7 +174,7 @@ func TestTaskImportResumable(t *testing.T) { assert.Equal(t, true, taskImporter.AllBatchesSubmitted()) workerPool.Wait() - err = tdb.QueryRow("SELECT count(*) FROM test_table").Scan(&rowCount) + err = tdb.QueryRow("SELECT count(*) FROM test_table_resume").Scan(&rowCount) assert.NoError(t, err) assert.Equal(t, int64(4), rowCount) } From 309995bd490bc82ae0eb17a6425c8669f5cbdb3b Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 18:44:39 +0530 Subject: [PATCH 32/87] readrestart retry --- .../test/containers/yugabytedb_container.go | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/yb-voyager/test/containers/yugabytedb_container.go b/yb-voyager/test/containers/yugabytedb_container.go index 822a7f6945..420b6c4941 100644 --- a/yb-voyager/test/containers/yugabytedb_container.go +++ b/yb-voyager/test/containers/yugabytedb_container.go @@ -4,10 +4,12 @@ import ( "context" "fmt" "os" + "strings" "time" "github.com/docker/go-connections/nat" "github.com/jackc/pgx/v5" + "github.com/samber/lo" log "github.com/sirupsen/logrus" "github.com/testcontainers/testcontainers-go" "github.com/testcontainers/testcontainers-go/wait" @@ -120,10 +122,25 @@ func (yb *YugabyteDBContainer) ExecuteSqls(sqls ...string) { } defer conn.Close(context.Background()) + retryCount := 3 + retryErrors := []string{ + "Restart read required at", + } for _, sql := range sqls { - _, err := conn.Exec(context.Background(), sql) + var err error + for i := 0; i < retryCount; i++ { + _, err = conn.Exec(context.Background(), sql) + if err == nil { + break + } + if !lo.ContainsBy(retryErrors, func(r string) bool { + return strings.Contains(err.Error(), r) + }) { + break + } + } if err != nil { - utils.ErrExit("failed to execute sql '%s': %w", sql, err) + utils.ErrExit("failed to execute sql: %v", err) } } } From ff9eb7626b08db32d114ff6ebe7a05814f9ca765 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 23 Jan 2025 18:50:20 +0530 Subject: [PATCH 33/87] sleep between retry --- yb-voyager/test/containers/yugabytedb_container.go | 1 + 1 file changed, 1 insertion(+) diff --git a/yb-voyager/test/containers/yugabytedb_container.go b/yb-voyager/test/containers/yugabytedb_container.go index 420b6c4941..0538a1f6d7 100644 --- a/yb-voyager/test/containers/yugabytedb_container.go +++ b/yb-voyager/test/containers/yugabytedb_container.go @@ -138,6 +138,7 @@ func (yb *YugabyteDBContainer) ExecuteSqls(sqls ...string) { }) { break } + time.Sleep(2 * time.Second) } if err != nil { utils.ErrExit("failed to execute sql: %v", err) From 6d98ebe39b422f90f6a5c53de5feb7325cc01475 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Fri, 24 Jan 2025 10:49:02 +0530 Subject: [PATCH 34/87] cleanup --- yb-voyager/cmd/importData.go | 68 ----------------- yb-voyager/cmd/importDataFileTaskImporter.go | 78 ++++++++++---------- 2 files changed, 39 insertions(+), 107 deletions(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index 8d2f3470b2..4599e6eb46 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -595,26 +595,6 @@ func importData(importFileTasks []*ImportFileTask) { utils.ErrExit("Failed to create file task importer: %s", err) } - // totalProgressAmount := getTotalProgressAmount(task) - // progressReporter.ImportFileStarted(task, totalProgressAmount) - // importedProgressAmount := getImportedProgressAmount(task, state) - // progressReporter.AddProgressAmount(task, importedProgressAmount) - - // var currentProgress int64 - // updateProgressFn := func(progressAmount int64) { - // currentProgress += progressAmount - // progressReporter.AddProgressAmount(task, progressAmount) - - // if importerRole == TARGET_DB_IMPORTER_ROLE && totalProgressAmount > currentProgress { - // importDataTableMetrics := createImportDataTableMetrics(task.TableNameTup.ForKey(), - // currentProgress, totalProgressAmount, ROW_UPDATE_STATUS_IN_PROGRESS) - // // The metrics are sent after evry 5 secs in implementation of UpdateImportedRowCount - // controlPlane.UpdateImportedRowCount( - // []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) - // } - // } - - // importFile(state, task, updateProgressFn) for !taskImporter.AllBatchesSubmitted() { err := taskImporter.SubmitNextBatch() if err != nil { @@ -624,14 +604,6 @@ func importData(importFileTasks []*ImportFileTask) { batchImportPool.Wait() // Wait for the file import to finish. taskImporter.PostProcess() - // if importerRole == TARGET_DB_IMPORTER_ROLE { - // importDataTableMetrics := createImportDataTableMetrics(task.TableNameTup.ForKey(), - // currentProgress, totalProgressAmount, ROW_UPDATE_STATUS_COMPLETED) - // controlPlane.UpdateImportedRowCount( - // []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) - // } - - // progressReporter.FileImportDone(task) // Remove the progress-bar for the file.\ } time.Sleep(time.Second * 2) } @@ -961,31 +933,6 @@ func cleanImportState(state *ImportDataState, tasks []*ImportFileTask) { } } -// func importFile(state *ImportDataState, task *ImportFileTask, updateProgressFn func(int64)) { - -// origDataFile := task.FilePath -// importBatchArgsProto := getImportBatchArgsProto(task.TableNameTup, task.FilePath) -// log.Infof("Start splitting table %q: data-file: %q", task.TableNameTup, origDataFile) - -// err := state.PrepareForFileImport(task.FilePath, task.TableNameTup) -// if err != nil { -// utils.ErrExit("preparing for file import: %s", err) -// } - -// fileBatchProducer, err := NewFileBatchProducer(task, state) -// if err != nil { -// utils.ErrExit("creating file batch producer: %s", err) -// } - -// for !fileBatchProducer.Done() { -// batch, err := fileBatchProducer.NextBatch() -// if err != nil { -// utils.ErrExit("getting next batch: %s", err) -// } -// submitBatch(batch, updateProgressFn, importBatchArgsProto) -// } -// } - func executePostSnapshotImportSqls() error { sequenceFilePath := filepath.Join(exportDir, "data", "postdata.sql") if utils.FileOrFolderExists(sequenceFilePath) { @@ -998,21 +945,6 @@ func executePostSnapshotImportSqls() error { return nil } -// func submitBatch(batch *Batch, updateProgressFn func(int64), importBatchArgsProto *tgtdb.ImportBatchArgs) { -// batchImportPool.Go(func() { -// // There are `poolSize` number of competing go-routines trying to invoke COPY. -// // But the `connPool` will allow only `parallelism` number of connections to be -// // used at a time. Thus limiting the number of concurrent COPYs to `parallelism`. -// importBatch(batch, importBatchArgsProto) -// if reportProgressInBytes { -// updateProgressFn(batch.ByteCount) -// } else { -// updateProgressFn(batch.RecordCount) -// } -// }) -// log.Infof("Queued batch: %s", spew.Sdump(batch)) -// } - func getIndexName(sqlQuery string, indexName string) (string, error) { // Return the index name itself if it is aleady qualified with schema name if len(strings.Split(indexName, ".")) == 2 { diff --git a/yb-voyager/cmd/importDataFileTaskImporter.go b/yb-voyager/cmd/importDataFileTaskImporter.go index 2b5834e3b8..a4d63554c9 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter.go +++ b/yb-voyager/cmd/importDataFileTaskImporter.go @@ -89,12 +89,50 @@ func (fti *FileTaskImporter) SubmitNextBatch() error { return fti.submitBatch(batch) } +func (fti *FileTaskImporter) importBatch(batch *Batch) { + err := batch.MarkPending() + if err != nil { + utils.ErrExit("marking batch as pending: %d: %s", batch.Number, err) + } + log.Infof("Importing %q", batch.FilePath) + + importBatchArgs := *fti.importBatchArgsProto + importBatchArgs.FilePath = batch.FilePath + importBatchArgs.RowsPerTransaction = batch.OffsetEnd - batch.OffsetStart + + var rowsAffected int64 + sleepIntervalSec := 0 + for attempt := 0; attempt < COPY_MAX_RETRY_COUNT; attempt++ { + tableSchema, _ := TableNameToSchema.Get(batch.TableNameTup) + rowsAffected, err = tdb.ImportBatch(batch, &importBatchArgs, exportDir, tableSchema) + if err == nil || tdb.IsNonRetryableCopyError(err) { + break + } + log.Warnf("COPY FROM file %q: %s", batch.FilePath, err) + sleepIntervalSec += 10 + if sleepIntervalSec > MAX_SLEEP_SECOND { + sleepIntervalSec = MAX_SLEEP_SECOND + } + log.Infof("sleep for %d seconds before retrying the file %s (attempt %d)", + sleepIntervalSec, batch.FilePath, attempt) + time.Sleep(time.Duration(sleepIntervalSec) * time.Second) + } + log.Infof("%q => %d rows affected", batch.FilePath, rowsAffected) + if err != nil { + utils.ErrExit("import batch: %q into %s: %s", batch.FilePath, batch.TableNameTup, err) + } + err = batch.MarkDone() + if err != nil { + utils.ErrExit("marking batch as done: %q: %s", batch.FilePath, err) + } +} + func (fti *FileTaskImporter) submitBatch(batch *Batch) error { fti.workerPool.Go(func() { // There are `poolSize` number of competing go-routines trying to invoke COPY. // But the `connPool` will allow only `parallelism` number of connections to be // used at a time. Thus limiting the number of concurrent COPYs to `parallelism`. - importBatch(batch, fti.importBatchArgsProto) + fti.importBatch(batch) if reportProgressInBytes { fti.updateProgress(batch.ByteCount) } else { @@ -207,41 +245,3 @@ func getImportBatchArgsProto(tableNameTup sqlname.NameTuple, filePath string) *t log.Infof("ImportBatchArgs: %v", spew.Sdump(importBatchArgsProto)) return importBatchArgsProto } - -func importBatch(batch *Batch, importBatchArgsProto *tgtdb.ImportBatchArgs) { - err := batch.MarkPending() - if err != nil { - utils.ErrExit("marking batch as pending: %d: %s", batch.Number, err) - } - log.Infof("Importing %q", batch.FilePath) - - importBatchArgs := *importBatchArgsProto - importBatchArgs.FilePath = batch.FilePath - importBatchArgs.RowsPerTransaction = batch.OffsetEnd - batch.OffsetStart - - var rowsAffected int64 - sleepIntervalSec := 0 - for attempt := 0; attempt < COPY_MAX_RETRY_COUNT; attempt++ { - tableSchema, _ := TableNameToSchema.Get(batch.TableNameTup) - rowsAffected, err = tdb.ImportBatch(batch, &importBatchArgs, exportDir, tableSchema) - if err == nil || tdb.IsNonRetryableCopyError(err) { - break - } - log.Warnf("COPY FROM file %q: %s", batch.FilePath, err) - sleepIntervalSec += 10 - if sleepIntervalSec > MAX_SLEEP_SECOND { - sleepIntervalSec = MAX_SLEEP_SECOND - } - log.Infof("sleep for %d seconds before retrying the file %s (attempt %d)", - sleepIntervalSec, batch.FilePath, attempt) - time.Sleep(time.Duration(sleepIntervalSec) * time.Second) - } - log.Infof("%q => %d rows affected", batch.FilePath, rowsAffected) - if err != nil { - utils.ErrExit("import batch: %q into %s: %s", batch.FilePath, batch.TableNameTup, err) - } - err = batch.MarkDone() - if err != nil { - utils.ErrExit("marking batch as done: %q: %s", batch.FilePath, err) - } -} From a3ec6881c567c00097a33e77c106437a857c62dc Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Fri, 24 Jan 2025 10:50:34 +0530 Subject: [PATCH 35/87] close batch producer --- yb-voyager/cmd/importDataFileBatchProducer.go | 4 ++++ yb-voyager/cmd/importDataFileTaskImporter.go | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer.go b/yb-voyager/cmd/importDataFileBatchProducer.go index 8551089cd6..f71f91ee90 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer.go +++ b/yb-voyager/cmd/importDataFileBatchProducer.go @@ -234,3 +234,7 @@ func (p *FileBatchProducer) finalizeBatch(batchWriter *BatchWriter, isLastBatch p.lastBatchNumber = batchNum return batch, nil } + +func (p *FileBatchProducer) Close() { + p.dataFile.Close() +} diff --git a/yb-voyager/cmd/importDataFileTaskImporter.go b/yb-voyager/cmd/importDataFileTaskImporter.go index a4d63554c9..1ae8ed6fd7 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter.go +++ b/yb-voyager/cmd/importDataFileTaskImporter.go @@ -157,7 +157,7 @@ func (fti *FileTaskImporter) updateProgress(progressAmount int64) { } func (fti *FileTaskImporter) PostProcess() { - // TODO: close fileBatchProducer properly + fti.batchProducer.Close() if importerRole == TARGET_DB_IMPORTER_ROLE { importDataTableMetrics := createImportDataTableMetrics(fti.task.TableNameTup.ForKey(), From 118418a37d34ed70375853814e79361cdeb4ae56 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Fri, 24 Jan 2025 10:52:35 +0530 Subject: [PATCH 36/87] clean up test --- yb-voyager/cmd/importDataFileTaskImporter_test.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go index add623f50f..efefa2e964 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter_test.go +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -39,7 +39,6 @@ func TestBasicTaskImport(t *testing.T) { } setupYugabyteTestDb(t) defer testYugabyteDBTarget.Finalize() - // defer testYugabyteDBTarget.TestContainer.Terminate(context.Background()) testYugabyteDBTarget.TestContainer.ExecuteSqls( `CREATE TABLE test_table_basic (id INT PRIMARY KEY, val TEXT);`, ) @@ -81,7 +80,6 @@ func TestImportAllBatchesAndResume(t *testing.T) { } setupYugabyteTestDb(t) defer testYugabyteDBTarget.Finalize() - // defer testYugabyteDBTarget.TestContainer.Terminate(context.Background()) testYugabyteDBTarget.TestContainer.ExecuteSqls( `CREATE TABLE test_table_all (id INT PRIMARY KEY, val TEXT);`, ) @@ -131,7 +129,6 @@ func TestTaskImportResumable(t *testing.T) { } setupYugabyteTestDb(t) defer testYugabyteDBTarget.Finalize() - // defer testYugabyteDBTarget.TestContainer.Terminate(context.Background()) testYugabyteDBTarget.TestContainer.ExecuteSqls( `CREATE TABLE test_table_resume (id INT PRIMARY KEY, val TEXT);`, ) @@ -155,14 +152,14 @@ func TestTaskImportResumable(t *testing.T) { err = taskImporter.SubmitNextBatch() assert.NoError(t, err) - // simulate restart - workerPool.Wait() // check that the first batch was imported + workerPool.Wait() var rowCount int64 err = tdb.QueryRow("SELECT count(*) FROM test_table_resume").Scan(&rowCount) assert.NoError(t, err) assert.Equal(t, int64(2), rowCount) + // simulate restart progressReporter = NewImportDataProgressReporter(true) workerPool = pool.New().WithMaxGoroutines(2) taskImporter, err = NewFileTaskImporter(task, state, workerPool, progressReporter) From 9dd04bec8cdedb6790204364f0e768f6e685c236 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Fri, 24 Jan 2025 13:08:25 +0530 Subject: [PATCH 37/87] reduce max restarts --- .../tests/resumption/import-file/single-large-table/config.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/migtests/tests/resumption/import-file/single-large-table/config.yaml b/migtests/tests/resumption/import-file/single-large-table/config.yaml index 025686e1a9..3f7d702154 100644 --- a/migtests/tests/resumption/import-file/single-large-table/config.yaml +++ b/migtests/tests/resumption/import-file/single-large-table/config.yaml @@ -14,7 +14,7 @@ row_count: # Resumption Settings resumption: - max_restarts: 30 + max_restarts: 20 min_interrupt_seconds: 300 max_interrupt_seconds: 720 min_restart_wait_seconds: 30 From b9b87cf51dbb31097e56be3e0a28d440f5b14d74 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Fri, 24 Jan 2025 14:17:28 +0530 Subject: [PATCH 38/87] if check before closing --- yb-voyager/cmd/importDataFileBatchProducer.go | 4 +++- yb-voyager/cmd/importDataFileTaskImporter.go | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/yb-voyager/cmd/importDataFileBatchProducer.go b/yb-voyager/cmd/importDataFileBatchProducer.go index f71f91ee90..43ef93edb4 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer.go +++ b/yb-voyager/cmd/importDataFileBatchProducer.go @@ -236,5 +236,7 @@ func (p *FileBatchProducer) finalizeBatch(batchWriter *BatchWriter, isLastBatch } func (p *FileBatchProducer) Close() { - p.dataFile.Close() + if p.dataFile != nil { + p.dataFile.Close() + } } diff --git a/yb-voyager/cmd/importDataFileTaskImporter.go b/yb-voyager/cmd/importDataFileTaskImporter.go index 1ae8ed6fd7..d5912b5585 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter.go +++ b/yb-voyager/cmd/importDataFileTaskImporter.go @@ -157,7 +157,9 @@ func (fti *FileTaskImporter) updateProgress(progressAmount int64) { } func (fti *FileTaskImporter) PostProcess() { - fti.batchProducer.Close() + if fti.batchProducer != nil { + fti.batchProducer.Close() + } if importerRole == TARGET_DB_IMPORTER_ROLE { importDataTableMetrics := createImportDataTableMetrics(fti.task.TableNameTup.ForKey(), From be9ca49a0602ff5582bed8f636ea7644ea2ce71c Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 27 Jan 2025 11:31:04 +0530 Subject: [PATCH 39/87] task picker initial commit --- yb-voyager/cmd/importData.go | 70 ++++++++++++++---- yb-voyager/cmd/importDataFileTaskImporter.go | 9 ++- yb-voyager/cmd/importDataFileTaskPicker.go | 76 ++++++++++++++++++++ 3 files changed, 139 insertions(+), 16 deletions(-) create mode 100644 yb-voyager/cmd/importDataFileTaskPicker.go diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index 4599e6eb46..ca956d8564 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -577,6 +577,10 @@ func importData(importFileTasks []*ImportFileTask) { } poolSize = yb.GetNumMaxConnectionsInPool() * 2 } + // The code can produce `poolSize` number of batches at a time. But, it can consume only + // `parallelism` number of batches at a time. + batchImportPool = pool.New().WithMaxGoroutines(poolSize) + log.Infof("created batch import pool of size: %d", poolSize) progressReporter := NewImportDataProgressReporter(bool(disablePb)) if importerRole == TARGET_DB_IMPORTER_ROLE { @@ -584,27 +588,67 @@ func importData(importFileTasks []*ImportFileTask) { controlPlane.UpdateImportedRowCount(importDataAllTableMetrics) } - for _, task := range pendingTasks { - // The code can produce `poolSize` number of batches at a time. But, it can consume only - // `parallelism` number of batches at a time. - batchImportPool = pool.New().WithMaxGoroutines(poolSize) - log.Infof("created batch import pool of size: %d", poolSize) + taskPicker, err := NewSequentialTaskPicker(pendingTasks, state) + if err != nil { + utils.ErrExit("Failed to create task picker: %w", err) + } + taskImporters := map[int]*FileTaskImporter{} - taskImporter, err := NewFileTaskImporter(task, state, batchImportPool, progressReporter) + for taskPicker.HasMoreTasks() { + task, err := taskPicker.NextTask() if err != nil { - utils.ErrExit("Failed to create file task importer: %s", err) + utils.ErrExit("Failed to get next task: %w", err) } - - for !taskImporter.AllBatchesSubmitted() { - err := taskImporter.SubmitNextBatch() + var taskImporter *FileTaskImporter + var ok bool + taskImporter, ok = taskImporters[task.ID] + if !ok { + taskImporter, err = NewFileTaskImporter(task, state, batchImportPool, progressReporter) if err != nil { - utils.ErrExit("Failed to submit next batch: task:%v err: %s", task, err) + utils.ErrExit("Failed to create file task importer: %s", err) } + taskImporters[task.ID] = taskImporter } - batchImportPool.Wait() // Wait for the file import to finish. - taskImporter.PostProcess() + if taskImporter.AllBatchesSubmitted() { + // All batches for this task have been submitted. + // task could have been completed OR in progress + // in case task is done, we should inform task picker. + taskDone, err := taskImporter.AllBatchesImported() + if err != nil { + utils.ErrExit("Failed to check if all batches are imported: task: %v err :%w", task, err) + } + if taskDone { + taskImporter.PostProcess() + err = taskPicker.MarkTaskAsDone(task) + if err != nil { + utils.ErrExit("Failed to mark task as done: task: %v, err: %w", task, err) + } + } + continue + } + err = taskImporter.SubmitNextBatch() + if err != nil { + utils.ErrExit("Failed to submit next batch: task:%v err: %s", task, err) + } } + + // for _, task := range pendingTasks { + // taskImporter, err := NewFileTaskImporter(task, state, batchImportPool, progressReporter) + // if err != nil { + // utils.ErrExit("Failed to create file task importer: %s", err) + // } + + // for !taskImporter.AllBatchesSubmitted() { + // err := taskImporter.SubmitNextBatch() + // if err != nil { + // utils.ErrExit("Failed to submit next batch: task:%v err: %s", task, err) + // } + // } + + // taskImporter.PostProcess() + // } + batchImportPool.Wait() // just ensure all tasks are done. time.Sleep(time.Second * 2) } utils.PrintAndLog("snapshot data import complete\n\n") diff --git a/yb-voyager/cmd/importDataFileTaskImporter.go b/yb-voyager/cmd/importDataFileTaskImporter.go index d5912b5585..57b29f1c4c 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter.go +++ b/yb-voyager/cmd/importDataFileTaskImporter.go @@ -73,9 +73,12 @@ func (fti *FileTaskImporter) AllBatchesSubmitted() bool { return fti.batchProducer.Done() } -func (fti *FileTaskImporter) AllBatchesImported() error { - // TODO: check importDataState for status. - panic("not implemented") +func (fti *FileTaskImporter) AllBatchesImported() (bool, error) { + taskStatus, err := fti.state.GetFileImportState(fti.task.FilePath, fti.task.TableNameTup) + if err != nil { + return false, fmt.Errorf("getting file import state: %s", err) + } + return taskStatus == FILE_IMPORT_COMPLETED, nil } func (fti *FileTaskImporter) SubmitNextBatch() error { diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go new file mode 100644 index 0000000000..bebe3d8638 --- /dev/null +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -0,0 +1,76 @@ +/* +Copyright (c) YugabyteDB, Inc. + +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 cmd + +import ( + "fmt" +) + +type FileTaskPicker interface { + NextTask() (*ImportFileTask, error) + MarkTaskAsDone(task *ImportFileTask) error + HasMoreTasks() bool +} + +type SequentialTaskPicker struct { + pendingTasks []*ImportFileTask + doneTasks []*ImportFileTask +} + +func NewSequentialTaskPicker(tasks []*ImportFileTask, state *ImportDataState) (*SequentialTaskPicker, error) { + var pendingTasks []*ImportFileTask + var doneTasks []*ImportFileTask + for _, task := range tasks { + taskStatus, err := state.GetFileImportState(task.FilePath, task.TableNameTup) + if err != nil { + return nil, fmt.Errorf("getting file import state for tasl: %v: %w", task, err) + } + switch taskStatus { + case FILE_IMPORT_COMPLETED: + doneTasks = append(doneTasks, task) + case FILE_IMPORT_NOT_STARTED, FILE_IMPORT_IN_PROGRESS: + pendingTasks = append(pendingTasks, task) + default: + return nil, fmt.Errorf("unexpected status for task: %v: %v", task, taskStatus) + } + } + return &SequentialTaskPicker{ + pendingTasks: pendingTasks, + doneTasks: doneTasks, + }, nil +} + +func (s *SequentialTaskPicker) NextTask() (*ImportFileTask, error) { + if !s.HasMoreTasks() { + return nil, fmt.Errorf("no more tasks") + } + return s.pendingTasks[0], nil +} + +func (s *SequentialTaskPicker) MarkTaskAsDone(task *ImportFileTask) error { + for i, t := range s.pendingTasks { + if t.ID == task.ID { + s.pendingTasks = append(s.pendingTasks[:i], s.pendingTasks[i+1:]...) + s.doneTasks = append(s.doneTasks, task) + return nil + } + } + return fmt.Errorf("task not found") +} + +func (s *SequentialTaskPicker) HasMoreTasks() bool { + return len(s.pendingTasks) > 0 +} From 653a3a451660a10720631578f458529bbb35a735 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 27 Jan 2025 12:18:22 +0530 Subject: [PATCH 40/87] run tests --- .github/workflows/go.yml | 2 +- .github/workflows/integration-tests.yml | 2 +- .github/workflows/issue-tests.yml | 2 +- .github/workflows/misc-migtests.yml | 2 +- .github/workflows/mysql-migtests.yml | 2 +- .github/workflows/pg-13-migtests.yml | 2 +- .github/workflows/pg-17-migtests.yml | 2 +- .github/workflows/pg-9-migtests.yml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index 88c20428d5..2d6a1c8098 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: [main] + branches: ['*'] jobs: diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index d94a594cb1..4d71e66fd3 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: [main] + branches: ['*'] env: ORACLE_INSTANT_CLIENT_VERSION: "21.5.0.0.0-1" diff --git a/.github/workflows/issue-tests.yml b/.github/workflows/issue-tests.yml index 185818624d..644925fd77 100644 --- a/.github/workflows/issue-tests.yml +++ b/.github/workflows/issue-tests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: [main] + branches: ['*'] jobs: diff --git a/.github/workflows/misc-migtests.yml b/.github/workflows/misc-migtests.yml index 053de031e4..69fbe4dffe 100644 --- a/.github/workflows/misc-migtests.yml +++ b/.github/workflows/misc-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-misc-migration-tests: diff --git a/.github/workflows/mysql-migtests.yml b/.github/workflows/mysql-migtests.yml index 2242861873..230a347ee2 100644 --- a/.github/workflows/mysql-migtests.yml +++ b/.github/workflows/mysql-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-mysql-migration-tests: diff --git a/.github/workflows/pg-13-migtests.yml b/.github/workflows/pg-13-migtests.yml index bcb78f4338..dd29300a2b 100644 --- a/.github/workflows/pg-13-migtests.yml +++ b/.github/workflows/pg-13-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-pg-13-migration-tests: diff --git a/.github/workflows/pg-17-migtests.yml b/.github/workflows/pg-17-migtests.yml index 771d20f8d1..0b4e702c42 100644 --- a/.github/workflows/pg-17-migtests.yml +++ b/.github/workflows/pg-17-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-pg-17-migration-tests: diff --git a/.github/workflows/pg-9-migtests.yml b/.github/workflows/pg-9-migtests.yml index 7fb10ef6e2..2c32ced464 100644 --- a/.github/workflows/pg-9-migtests.yml +++ b/.github/workflows/pg-9-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['main'] + branches: ['*'] jobs: run-pg-9-migration-tests: From d9d96996dfbaaab5240142ed7db41e4a79dcd5f5 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 27 Jan 2025 12:20:54 +0530 Subject: [PATCH 41/87] run tests 2 --- .github/workflows/go.yml | 2 +- .github/workflows/integration-tests.yml | 2 +- .github/workflows/issue-tests.yml | 2 +- .github/workflows/misc-migtests.yml | 2 +- .github/workflows/mysql-migtests.yml | 2 +- .github/workflows/pg-13-migtests.yml | 2 +- .github/workflows/pg-17-migtests.yml | 2 +- .github/workflows/pg-9-migtests.yml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index 2d6a1c8098..e2794811e7 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index 4d71e66fd3..af9410ed55 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/issue-tests.yml b/.github/workflows/issue-tests.yml index 644925fd77..2fc2307257 100644 --- a/.github/workflows/issue-tests.yml +++ b/.github/workflows/issue-tests.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/misc-migtests.yml b/.github/workflows/misc-migtests.yml index 69fbe4dffe..37dd3c0492 100644 --- a/.github/workflows/misc-migtests.yml +++ b/.github/workflows/misc-migtests.yml @@ -2,7 +2,7 @@ name: "Misc: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/mysql-migtests.yml b/.github/workflows/mysql-migtests.yml index 230a347ee2..04ce8bf52f 100644 --- a/.github/workflows/mysql-migtests.yml +++ b/.github/workflows/mysql-migtests.yml @@ -2,7 +2,7 @@ name: "MySQL: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-13-migtests.yml b/.github/workflows/pg-13-migtests.yml index dd29300a2b..b761072bf4 100644 --- a/.github/workflows/pg-13-migtests.yml +++ b/.github/workflows/pg-13-migtests.yml @@ -2,7 +2,7 @@ name: "PG 13: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-17-migtests.yml b/.github/workflows/pg-17-migtests.yml index 0b4e702c42..695a873dc1 100644 --- a/.github/workflows/pg-17-migtests.yml +++ b/.github/workflows/pg-17-migtests.yml @@ -2,7 +2,7 @@ name: "PG 17: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-9-migtests.yml b/.github/workflows/pg-9-migtests.yml index 2c32ced464..1647827243 100644 --- a/.github/workflows/pg-9-migtests.yml +++ b/.github/workflows/pg-9-migtests.yml @@ -2,7 +2,7 @@ name: "PG 9: Migration Tests" on: push: - branches: ['main', '*.*-dev', '*.*.*-dev'] + branches: ['*'] pull_request: branches: ['*'] From b08d7afc9874cc9685367d8e9e2fcd418b4303b1 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 27 Jan 2025 12:33:13 +0530 Subject: [PATCH 42/87] empty From c83aa51764a2374a919faeb756ee9412f3adf82a Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 27 Jan 2025 12:33:54 +0530 Subject: [PATCH 43/87] Revert "run tests 2" This reverts commit d9d96996dfbaaab5240142ed7db41e4a79dcd5f5. --- .github/workflows/go.yml | 2 +- .github/workflows/integration-tests.yml | 2 +- .github/workflows/issue-tests.yml | 2 +- .github/workflows/misc-migtests.yml | 2 +- .github/workflows/mysql-migtests.yml | 2 +- .github/workflows/pg-13-migtests.yml | 2 +- .github/workflows/pg-17-migtests.yml | 2 +- .github/workflows/pg-9-migtests.yml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index e2794811e7..2d6a1c8098 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index af9410ed55..4d71e66fd3 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/issue-tests.yml b/.github/workflows/issue-tests.yml index 2fc2307257..644925fd77 100644 --- a/.github/workflows/issue-tests.yml +++ b/.github/workflows/issue-tests.yml @@ -2,7 +2,7 @@ name: Go on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/misc-migtests.yml b/.github/workflows/misc-migtests.yml index 37dd3c0492..69fbe4dffe 100644 --- a/.github/workflows/misc-migtests.yml +++ b/.github/workflows/misc-migtests.yml @@ -2,7 +2,7 @@ name: "Misc: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/mysql-migtests.yml b/.github/workflows/mysql-migtests.yml index 04ce8bf52f..230a347ee2 100644 --- a/.github/workflows/mysql-migtests.yml +++ b/.github/workflows/mysql-migtests.yml @@ -2,7 +2,7 @@ name: "MySQL: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-13-migtests.yml b/.github/workflows/pg-13-migtests.yml index b761072bf4..dd29300a2b 100644 --- a/.github/workflows/pg-13-migtests.yml +++ b/.github/workflows/pg-13-migtests.yml @@ -2,7 +2,7 @@ name: "PG 13: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-17-migtests.yml b/.github/workflows/pg-17-migtests.yml index 695a873dc1..0b4e702c42 100644 --- a/.github/workflows/pg-17-migtests.yml +++ b/.github/workflows/pg-17-migtests.yml @@ -2,7 +2,7 @@ name: "PG 17: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] diff --git a/.github/workflows/pg-9-migtests.yml b/.github/workflows/pg-9-migtests.yml index 1647827243..2c32ced464 100644 --- a/.github/workflows/pg-9-migtests.yml +++ b/.github/workflows/pg-9-migtests.yml @@ -2,7 +2,7 @@ name: "PG 9: Migration Tests" on: push: - branches: ['*'] + branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: branches: ['*'] From d31a434bbe94888720f53638e93d31a4bb1d53a2 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 27 Jan 2025 12:34:06 +0530 Subject: [PATCH 44/87] Revert "run tests" This reverts commit 653a3a451660a10720631578f458529bbb35a735. --- .github/workflows/go.yml | 2 +- .github/workflows/integration-tests.yml | 2 +- .github/workflows/issue-tests.yml | 2 +- .github/workflows/misc-migtests.yml | 2 +- .github/workflows/mysql-migtests.yml | 2 +- .github/workflows/pg-13-migtests.yml | 2 +- .github/workflows/pg-17-migtests.yml | 2 +- .github/workflows/pg-9-migtests.yml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index 2d6a1c8098..88c20428d5 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: [main] jobs: diff --git a/.github/workflows/integration-tests.yml b/.github/workflows/integration-tests.yml index 4d71e66fd3..d94a594cb1 100644 --- a/.github/workflows/integration-tests.yml +++ b/.github/workflows/integration-tests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: [main] env: ORACLE_INSTANT_CLIENT_VERSION: "21.5.0.0.0-1" diff --git a/.github/workflows/issue-tests.yml b/.github/workflows/issue-tests.yml index 644925fd77..185818624d 100644 --- a/.github/workflows/issue-tests.yml +++ b/.github/workflows/issue-tests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: [main] jobs: diff --git a/.github/workflows/misc-migtests.yml b/.github/workflows/misc-migtests.yml index 69fbe4dffe..053de031e4 100644 --- a/.github/workflows/misc-migtests.yml +++ b/.github/workflows/misc-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-misc-migration-tests: diff --git a/.github/workflows/mysql-migtests.yml b/.github/workflows/mysql-migtests.yml index 230a347ee2..2242861873 100644 --- a/.github/workflows/mysql-migtests.yml +++ b/.github/workflows/mysql-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-mysql-migration-tests: diff --git a/.github/workflows/pg-13-migtests.yml b/.github/workflows/pg-13-migtests.yml index dd29300a2b..bcb78f4338 100644 --- a/.github/workflows/pg-13-migtests.yml +++ b/.github/workflows/pg-13-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-pg-13-migration-tests: diff --git a/.github/workflows/pg-17-migtests.yml b/.github/workflows/pg-17-migtests.yml index 0b4e702c42..771d20f8d1 100644 --- a/.github/workflows/pg-17-migtests.yml +++ b/.github/workflows/pg-17-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-pg-17-migration-tests: diff --git a/.github/workflows/pg-9-migtests.yml b/.github/workflows/pg-9-migtests.yml index 2c32ced464..7fb10ef6e2 100644 --- a/.github/workflows/pg-9-migtests.yml +++ b/.github/workflows/pg-9-migtests.yml @@ -4,7 +4,7 @@ on: push: branches: ['main', '*.*-dev', '*.*.*-dev'] pull_request: - branches: ['*'] + branches: ['main'] jobs: run-pg-9-migration-tests: From a4bd311b7c1bec114756f417a06ab8f1ead46ab0 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 27 Jan 2025 13:37:16 +0530 Subject: [PATCH 45/87] minor refactor + bug fix --- migtests/tests/import-file/validate | 4 +- yb-voyager/cmd/importData.go | 135 ++++++++++++++---------- yb-voyager/cmd/importDataFileCommand.go | 6 +- 3 files changed, 87 insertions(+), 58 deletions(-) diff --git a/migtests/tests/import-file/validate b/migtests/tests/import-file/validate index 18b83348b5..21b3d11e35 100755 --- a/migtests/tests/import-file/validate +++ b/migtests/tests/import-file/validate @@ -132,7 +132,9 @@ TABLE_NAME_DATA_MAP = { def file_import_done_checks(tgt): fetched_row_cnts = tgt.row_count_of_all_tables() print(f"Row counts after import data file: {fetched_row_cnts}") - assert fetched_row_cnts == EXPECTED + for table_name, rc in EXPECTED.items(): + assert fetched_row_cnts[table_name] == rc, f"Row count mismatch for table {table_name}, expected: {rc}, got: {fetched_row_cnts[table_name]}" + fetched_row_cnts_non_public = tgt.row_count_of_all_tables("non_public") print(f"Row counts after import data file in non_public schema: {fetched_row_cnts_non_public}") diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index ca956d8564..f31d06be1c 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -155,6 +155,7 @@ func importDataCommandFn(cmd *cobra.Command, args []string) { // TODO: handle case-sensitive in table names with oracle ff-db // quoteTableNameIfRequired() importFileTasks := discoverFilesToImport() + log.Debugf("Discovered import file tasks: %v", importFileTasks) if importerRole == TARGET_DB_IMPORTER_ROLE { importType = record.ExportType @@ -316,6 +317,10 @@ type ImportFileTask struct { FileSize int64 } +func (task *ImportFileTask) String() string { + return fmt.Sprintf("{ID: %d, FilePath: %s, TableName: %s, RowCount: %d, FileSize: %d}", task.ID, task.FilePath, task.TableNameTup.ForOutput(), task.RowCount, task.FileSize) +} + // func quoteTableNameIfRequired() { // if tconf.TargetDBType != ORACLE { // return @@ -543,6 +548,8 @@ func importData(importFileTasks []*ImportFileTask) { utils.ErrExit("Failed to classify tasks: %s", err) } } + log.Debugf("pending tasks: %v", pendingTasks) + log.Debugf("completed tasks: %v", completedTasks) //TODO: BUG: we are applying table-list filter on importFileTasks, but here we are considering all tables as per // export-data table-list. Should be fine because we are only disabling and re-enabling, but this is still not ideal. @@ -577,10 +584,6 @@ func importData(importFileTasks []*ImportFileTask) { } poolSize = yb.GetNumMaxConnectionsInPool() * 2 } - // The code can produce `poolSize` number of batches at a time. But, it can consume only - // `parallelism` number of batches at a time. - batchImportPool = pool.New().WithMaxGoroutines(poolSize) - log.Infof("created batch import pool of size: %d", poolSize) progressReporter := NewImportDataProgressReporter(bool(disablePb)) if importerRole == TARGET_DB_IMPORTER_ROLE { @@ -588,68 +591,36 @@ func importData(importFileTasks []*ImportFileTask) { controlPlane.UpdateImportedRowCount(importDataAllTableMetrics) } - taskPicker, err := NewSequentialTaskPicker(pendingTasks, state) - if err != nil { - utils.ErrExit("Failed to create task picker: %w", err) - } - taskImporters := map[int]*FileTaskImporter{} - - for taskPicker.HasMoreTasks() { - task, err := taskPicker.NextTask() + useTaskPicker := utils.GetEnvAsBool("USE_TASK_PICKER_FOR_IMPORT", true) + if useTaskPicker { + err := importTasksViaTaskPicker(pendingTasks, state, progressReporter, poolSize) if err != nil { - utils.ErrExit("Failed to get next task: %w", err) + utils.ErrExit("Failed to import tasks via task picker: %s", err) } - var taskImporter *FileTaskImporter - var ok bool - taskImporter, ok = taskImporters[task.ID] - if !ok { - taskImporter, err = NewFileTaskImporter(task, state, batchImportPool, progressReporter) + } else { + for _, task := range pendingTasks { + // The code can produce `poolSize` number of batches at a time. But, it can consume only + // `parallelism` number of batches at a time. + batchImportPool = pool.New().WithMaxGoroutines(poolSize) + log.Infof("created batch import pool of size: %d", poolSize) + + taskImporter, err := NewFileTaskImporter(task, state, batchImportPool, progressReporter) if err != nil { utils.ErrExit("Failed to create file task importer: %s", err) } - taskImporters[task.ID] = taskImporter - } - if taskImporter.AllBatchesSubmitted() { - // All batches for this task have been submitted. - // task could have been completed OR in progress - // in case task is done, we should inform task picker. - taskDone, err := taskImporter.AllBatchesImported() - if err != nil { - utils.ErrExit("Failed to check if all batches are imported: task: %v err :%w", task, err) - } - if taskDone { - taskImporter.PostProcess() - err = taskPicker.MarkTaskAsDone(task) + for !taskImporter.AllBatchesSubmitted() { + err := taskImporter.SubmitNextBatch() if err != nil { - utils.ErrExit("Failed to mark task as done: task: %v, err: %w", task, err) + utils.ErrExit("Failed to submit next batch: task:%v err: %s", task, err) } } - continue - } - err = taskImporter.SubmitNextBatch() - if err != nil { - utils.ErrExit("Failed to submit next batch: task:%v err: %s", task, err) + + taskImporter.PostProcess() } + batchImportPool.Wait() // just ensure all tasks are done. + time.Sleep(time.Second * 2) } - - // for _, task := range pendingTasks { - // taskImporter, err := NewFileTaskImporter(task, state, batchImportPool, progressReporter) - // if err != nil { - // utils.ErrExit("Failed to create file task importer: %s", err) - // } - - // for !taskImporter.AllBatchesSubmitted() { - // err := taskImporter.SubmitNextBatch() - // if err != nil { - // utils.ErrExit("Failed to submit next batch: task:%v err: %s", task, err) - // } - // } - - // taskImporter.PostProcess() - // } - batchImportPool.Wait() // just ensure all tasks are done. - time.Sleep(time.Second * 2) } utils.PrintAndLog("snapshot data import complete\n\n") } @@ -728,6 +699,60 @@ func importData(importFileTasks []*ImportFileTask) { } +func importTasksViaTaskPicker(pendingTasks []*ImportFileTask, state *ImportDataState, progressReporter *ImportDataProgressReporter, poolSize int) error { + // The code can produce `poolSize` number of batches at a time. But, it can consume only + // `parallelism` number of batches at a time. + batchImportPool = pool.New().WithMaxGoroutines(poolSize) + log.Infof("created batch import pool of size: %d", poolSize) + + taskPicker, err := NewSequentialTaskPicker(pendingTasks, state) + if err != nil { + return fmt.Errorf("create task picker: %w", err) + } + taskImporters := map[int]*FileTaskImporter{} + + for taskPicker.HasMoreTasks() { + task, err := taskPicker.NextTask() + if err != nil { + return fmt.Errorf("get next task: %w", err) + } + var taskImporter *FileTaskImporter + var ok bool + taskImporter, ok = taskImporters[task.ID] + if !ok { + taskImporter, err = NewFileTaskImporter(task, state, batchImportPool, progressReporter) + if err != nil { + return fmt.Errorf("create file task importer: %s", err) + } + log.Infof("created file task importer for table: %s, task: %v", task.TableNameTup.ForOutput(), task) + taskImporters[task.ID] = taskImporter + } + + if taskImporter.AllBatchesSubmitted() { + // All batches for this task have been submitted. + // task could have been completed (all batches imported) OR still in progress + // in case task is done, we should inform task picker so that we stop picking that task. + taskDone, err := taskImporter.AllBatchesImported() + if err != nil { + return fmt.Errorf("check if all batches are imported: task: %v err :%w", task, err) + } + if taskDone { + taskImporter.PostProcess() + err = taskPicker.MarkTaskAsDone(task) + if err != nil { + return fmt.Errorf("mark task as done: task: %v, err: %w", task, err) + } + } + continue + } + err = taskImporter.SubmitNextBatch() + if err != nil { + return fmt.Errorf("submit next batch: task:%v err: %s", task, err) + } + } + return nil +} + func startAdaptiveParallelism() (bool, error) { yb, ok := tdb.(*tgtdb.TargetYugabyteDB) if !ok { diff --git a/yb-voyager/cmd/importDataFileCommand.go b/yb-voyager/cmd/importDataFileCommand.go index 1e6aa10863..f9bb26982d 100644 --- a/yb-voyager/cmd/importDataFileCommand.go +++ b/yb-voyager/cmd/importDataFileCommand.go @@ -168,7 +168,8 @@ func prepareImportFileTasks() []*ImportFileTask { return result } kvs := strings.Split(fileTableMapping, ",") - for i, kv := range kvs { + idIndex := 0 + for _, kv := range kvs { globPattern, table := strings.Split(kv, ":")[0], strings.Split(kv, ":")[1] filePaths, err := dataStore.Glob(globPattern) if err != nil { @@ -187,12 +188,13 @@ func prepareImportFileTasks() []*ImportFileTask { utils.ErrExit("calculating file size in bytes: %q: %v", filePath, err) } task := &ImportFileTask{ - ID: i, + ID: idIndex, FilePath: filePath, TableNameTup: tableNameTuple, FileSize: fileSize, } result = append(result, task) + idIndex++ } } return result From 5213f07595e84d1cf2cfcb36b28fa233e4ab96fd Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 27 Jan 2025 13:42:15 +0530 Subject: [PATCH 46/87] better implementation for MarkTaskAsDone --- yb-voyager/cmd/importDataFileTaskPicker.go | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index bebe3d8638..5863ea9757 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -61,14 +61,17 @@ func (s *SequentialTaskPicker) NextTask() (*ImportFileTask, error) { } func (s *SequentialTaskPicker) MarkTaskAsDone(task *ImportFileTask) error { - for i, t := range s.pendingTasks { - if t.ID == task.ID { - s.pendingTasks = append(s.pendingTasks[:i], s.pendingTasks[i+1:]...) - s.doneTasks = append(s.doneTasks, task) - return nil - } + // it is assumed that the task is in pendingTasks and the first task in the list. + // because SequentialTaskPicker will always pick the first task from the list. + if !s.HasMoreTasks() { + return fmt.Errorf("no more pending tasks to mark as done") + } + if s.pendingTasks[0].ID != task.ID { + return fmt.Errorf("Task provided is not the first pending task. task's id = %d, first pending task's id = %d. ", task.ID, s.pendingTasks[0].ID) } - return fmt.Errorf("task not found") + s.pendingTasks = s.pendingTasks[1:] + s.doneTasks = append(s.doneTasks, task) + return nil } func (s *SequentialTaskPicker) HasMoreTasks() bool { From 520745cae820719cbbff8bc4a8358d6a652d4d83 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 28 Jan 2025 11:06:42 +0530 Subject: [PATCH 47/87] review ccomments --- yb-voyager/cmd/importDataFileTaskImporter.go | 18 +++--- .../cmd/importDataFileTaskImporter_test.go | 59 +++++++++++++++++++ 2 files changed, 69 insertions(+), 8 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskImporter.go b/yb-voyager/cmd/importDataFileTaskImporter.go index d5912b5585..c91738ebea 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter.go +++ b/yb-voyager/cmd/importDataFileTaskImporter.go @@ -50,16 +50,18 @@ func NewFileTaskImporter(task *ImportFileTask, state *ImportDataState, workerPoo } totalProgressAmount := getTotalProgressAmount(task) progressReporter.ImportFileStarted(task, totalProgressAmount) - progressReporter.AddProgressAmount(task, getImportedProgressAmount(task, state)) + currentProgressAmount := getImportedProgressAmount(task, state) + progressReporter.AddProgressAmount(task, currentProgressAmount) fti := &FileTaskImporter{ - task: task, - state: state, - batchProducer: batchProducer, - workerPool: workerPool, - importBatchArgsProto: getImportBatchArgsProto(task.TableNameTup, task.FilePath), - progressReporter: progressReporter, - totalProgressAmount: getTotalProgressAmount(task), + task: task, + state: state, + batchProducer: batchProducer, + workerPool: workerPool, + importBatchArgsProto: getImportBatchArgsProto(task.TableNameTup, task.FilePath), + progressReporter: progressReporter, + totalProgressAmount: totalProgressAmount, + currentProgressAmount: currentProgressAmount, } return fti, nil } diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go index efefa2e964..c28e740ec2 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter_test.go +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -175,3 +175,62 @@ func TestTaskImportResumable(t *testing.T) { assert.NoError(t, err) assert.Equal(t, int64(4), rowCount) } + +func TestTaskImportResumableNoPK(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + setupYugabyteTestDb(t) + defer testYugabyteDBTarget.Finalize() + testYugabyteDBTarget.TestContainer.ExecuteSqls( + `CREATE TABLE test_table_resume_no_pk (id INT, val TEXT);`, + ) + defer testYugabyteDBTarget.TestContainer.ExecuteSqls(`DROP TABLE test_table_resume_no_pk;`) + + // file import + fileContents := `id,val +1, "hello" +2, "world" +3, "foo" +4, "bar"` + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_resume_no_pk") + testutils.FatalIfError(t, err) + + progressReporter := NewImportDataProgressReporter(true) + workerPool := pool.New().WithMaxGoroutines(2) + taskImporter, err := NewFileTaskImporter(task, state, workerPool, progressReporter) + testutils.FatalIfError(t, err) + + // submit 1 batch + err = taskImporter.SubmitNextBatch() + assert.NoError(t, err) + + // check that the first batch was imported + workerPool.Wait() + var rowCount int64 + err = tdb.QueryRow("SELECT count(*) FROM test_table_resume_no_pk").Scan(&rowCount) + assert.NoError(t, err) + assert.Equal(t, int64(2), rowCount) + + // simulate restart + progressReporter = NewImportDataProgressReporter(true) + workerPool = pool.New().WithMaxGoroutines(2) + taskImporter, err = NewFileTaskImporter(task, state, workerPool, progressReporter) + testutils.FatalIfError(t, err) + + // submit second batch, not first batch again as it was already imported + err = taskImporter.SubmitNextBatch() + assert.NoError(t, err) + + assert.Equal(t, true, taskImporter.AllBatchesSubmitted()) + workerPool.Wait() + err = tdb.QueryRow("SELECT count(*) FROM test_table_resume_no_pk").Scan(&rowCount) + assert.NoError(t, err) + assert.Equal(t, int64(4), rowCount) +} From 8f8304ab16abdc4f618e5660ec67852c86ab2115 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 28 Jan 2025 11:11:30 +0530 Subject: [PATCH 48/87] review comments 2 --- yb-voyager/cmd/importDataFileTaskImporter.go | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskImporter.go b/yb-voyager/cmd/importDataFileTaskImporter.go index c91738ebea..e0e4aa9a9d 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter.go +++ b/yb-voyager/cmd/importDataFileTaskImporter.go @@ -149,12 +149,9 @@ func (fti *FileTaskImporter) updateProgress(progressAmount int64) { fti.currentProgressAmount += progressAmount fti.progressReporter.AddProgressAmount(fti.task, progressAmount) - if importerRole == TARGET_DB_IMPORTER_ROLE && fti.totalProgressAmount > fti.currentProgressAmount { - importDataTableMetrics := createImportDataTableMetrics(fti.task.TableNameTup.ForKey(), - fti.currentProgressAmount, fti.totalProgressAmount, ROW_UPDATE_STATUS_IN_PROGRESS) - // The metrics are sent after evry 5 secs in implementation of UpdateImportedRowCount - controlPlane.UpdateImportedRowCount( - []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) + // The metrics are sent after evry 5 secs in implementation of UpdateImportedRowCount + if fti.totalProgressAmount > fti.currentProgressAmount { + fti.updateProgressInControlPlane(ROW_UPDATE_STATUS_IN_PROGRESS) } } @@ -163,14 +160,18 @@ func (fti *FileTaskImporter) PostProcess() { fti.batchProducer.Close() } + fti.updateProgressInControlPlane(ROW_UPDATE_STATUS_COMPLETED) + + fti.progressReporter.FileImportDone(fti.task) // Remove the progress-bar for the file.\ +} + +func (fti *FileTaskImporter) updateProgressInControlPlane(status int) { if importerRole == TARGET_DB_IMPORTER_ROLE { importDataTableMetrics := createImportDataTableMetrics(fti.task.TableNameTup.ForKey(), - fti.currentProgressAmount, fti.totalProgressAmount, ROW_UPDATE_STATUS_COMPLETED) + fti.currentProgressAmount, fti.totalProgressAmount, status) controlPlane.UpdateImportedRowCount( []*cp.UpdateImportedRowCountEvent{&importDataTableMetrics}) } - - fti.progressReporter.FileImportDone(fti.task) // Remove the progress-bar for the file.\ } // ============================================================================= // From 71f37f70503a94b5f5d43a4d34d96cd5c15fdb88 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 28 Jan 2025 12:33:16 +0530 Subject: [PATCH 49/87] test cases --- .../cmd/importDataFileBatchProducer_test.go | 12 +- .../cmd/importDataFileTaskImporter_test.go | 8 +- .../cmd/importDataFileTaskPicker_test.go | 116 ++++++++++++++++++ yb-voyager/cmd/importDataTestUtils.go | 4 +- 4 files changed, 128 insertions(+), 12 deletions(-) create mode 100644 yb-voyager/cmd/importDataFileTaskPicker_test.go diff --git a/yb-voyager/cmd/importDataFileBatchProducer_test.go b/yb-voyager/cmd/importDataFileBatchProducer_test.go index 4b3f171105..bdb38e4fdf 100644 --- a/yb-voyager/cmd/importDataFileBatchProducer_test.go +++ b/yb-voyager/cmd/importDataFileBatchProducer_test.go @@ -39,7 +39,7 @@ func TestBasicFileBatchProducer(t *testing.T) { fileContents := `id,val 1, "hello"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table", 1) assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -71,7 +71,7 @@ func TestFileBatchProducerBasedOnRowsThreshold(t *testing.T) { 2, "world" 3, "foo" 4, "bar"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table", 1) assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -122,7 +122,7 @@ func TestFileBatchProducerBasedOnSizeThreshold(t *testing.T) { 2, "ghijk" 3, "mnopq" 4, "stuvw"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table", 1) assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -182,7 +182,7 @@ func TestFileBatchProducerThrowsErrorWhenSingleRowGreaterThanMaxBatchSize(t *tes 2, "ghijk" 3, "mnopq1234567899876543" 4, "stuvw"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table", 1) assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -218,7 +218,7 @@ func TestFileBatchProducerResumable(t *testing.T) { 2, "world" 3, "foo" 4, "bar"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table", 1) assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) @@ -274,7 +274,7 @@ func TestFileBatchProducerResumeAfterAllBatchesProduced(t *testing.T) { 2, "world" 3, "foo" 4, "bar"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table", 1) assert.NoError(t, err) batchproducer, err := NewFileBatchProducer(task, state) diff --git a/yb-voyager/cmd/importDataFileTaskImporter_test.go b/yb-voyager/cmd/importDataFileTaskImporter_test.go index c28e740ec2..09c84c3a97 100644 --- a/yb-voyager/cmd/importDataFileTaskImporter_test.go +++ b/yb-voyager/cmd/importDataFileTaskImporter_test.go @@ -48,7 +48,7 @@ func TestBasicTaskImport(t *testing.T) { fileContents := `id,val 1, "hello" 2, "world"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_basic") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_basic", 1) testutils.FatalIfError(t, err) progressReporter := NewImportDataProgressReporter(true) @@ -89,7 +89,7 @@ func TestImportAllBatchesAndResume(t *testing.T) { fileContents := `id,val 1, "hello" 2, "world"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_all") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_all", 1) testutils.FatalIfError(t, err) progressReporter := NewImportDataProgressReporter(true) @@ -140,7 +140,7 @@ func TestTaskImportResumable(t *testing.T) { 2, "world" 3, "foo" 4, "bar"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_resume") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_resume", 1) testutils.FatalIfError(t, err) progressReporter := NewImportDataProgressReporter(true) @@ -199,7 +199,7 @@ func TestTaskImportResumableNoPK(t *testing.T) { 2, "world" 3, "foo" 4, "bar"` - _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_resume_no_pk") + _, task, err := createFileAndTask(lexportDir, fileContents, ldataDir, "test_table_resume_no_pk", 1) testutils.FatalIfError(t, err) progressReporter := NewImportDataProgressReporter(true) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go new file mode 100644 index 0000000000..5d2826012f --- /dev/null +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -0,0 +1,116 @@ +/* +Copyright (c) YugabyteDB, Inc. + +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 cmd + +import ( + "fmt" + "os" + "testing" + + "github.com/stretchr/testify/assert" + testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" +) + +func TestSequentialTaskPickerBasic(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, task1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.table1", 1) + testutils.FatalIfError(t, err) + _, task2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.table2", 2) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + task1, + task2, + } + + picker, err := NewSequentialTaskPicker(tasks, state) + testutils.FatalIfError(t, err) + + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask, it should return the same task (first task) + for i := 0; i < 10; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Equal(t, task1, task) + } +} + +func TestSequentialTaskPickerMarkTaskDone(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, task1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.table1", 1) + testutils.FatalIfError(t, err) + _, task2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.table2", 2) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + task1, + task2, + } + + picker, err := NewSequentialTaskPicker(tasks, state) + testutils.FatalIfError(t, err) + + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask, it should return the same task (first task) + for i := 0; i < 10; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Equal(t, task1, task) + } + + // mark the second task as done; should return err + err = picker.MarkTaskAsDone(task2) + assert.Error(t, err) + + // mark the first task as done, now, the picker should return task2 + err = picker.MarkTaskAsDone(task1) + assert.NoError(t, err) + + for i := 0; i < 10; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Equal(t, task2, task) + } + + // mark the second task as done, then the picker should not have any tasks anymore + err = picker.MarkTaskAsDone(task2) + assert.NoError(t, err) + assert.False(t, picker.HasMoreTasks()) + + // marking any task as done now should return an error + err = picker.MarkTaskAsDone(task1) + assert.Error(t, err) +} diff --git a/yb-voyager/cmd/importDataTestUtils.go b/yb-voyager/cmd/importDataTestUtils.go index 6c6ef4cd56..2df858157f 100644 --- a/yb-voyager/cmd/importDataTestUtils.go +++ b/yb-voyager/cmd/importDataTestUtils.go @@ -99,7 +99,7 @@ func setupExportDirAndImportDependencies(batchSizeRows int64, batchSizeBytes int return ldataDir, lexportDir, state, nil } -func createFileAndTask(lexportDir string, fileContents string, ldataDir string, tableName string) (string, *ImportFileTask, error) { +func createFileAndTask(lexportDir string, fileContents string, ldataDir string, tableName string, id int) (string, *ImportFileTask, error) { dataFileDescriptor = &datafile.Descriptor{ FileFormat: "csv", Delimiter: ",", @@ -117,7 +117,7 @@ func createFileAndTask(lexportDir string, fileContents string, ldataDir string, sourceName := sqlname.NewObjectName(constants.POSTGRESQL, "public", "public", tableName) tableNameTup := sqlname.NameTuple{SourceName: sourceName, CurrentName: sourceName} task := &ImportFileTask{ - ID: 1, + ID: id, FilePath: tempFile, TableNameTup: tableNameTup, RowCount: 1, From 72684d805fb512a3eed1e1409c8c77f87bdd2f58 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 28 Jan 2025 12:40:47 +0530 Subject: [PATCH 50/87] fix old behaviour --- yb-voyager/cmd/importData.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index f31d06be1c..5d821cfc54 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -616,9 +616,9 @@ func importData(importFileTasks []*ImportFileTask) { } } + batchImportPool.Wait() // wait for file import to finish taskImporter.PostProcess() } - batchImportPool.Wait() // just ensure all tasks are done. time.Sleep(time.Second * 2) } } From 4bbbd0d41e5b2113ff7bb27132b68b78dee59c8f Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 28 Jan 2025 12:47:20 +0530 Subject: [PATCH 51/87] comment --- yb-voyager/cmd/importDataFileTaskPicker.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 5863ea9757..c8620e9795 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -25,6 +25,10 @@ type FileTaskPicker interface { HasMoreTasks() bool } +/* +A sequential task picker ensures that mulitple tasks are not being processed at the same time. +It will always pick the same task (first task in the pending list) until it is marked as done. +*/ type SequentialTaskPicker struct { pendingTasks []*ImportFileTask doneTasks []*ImportFileTask From 03373452a575e70eacdc4d463c6e3c01929c95a0 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 28 Jan 2025 14:52:07 +0530 Subject: [PATCH 52/87] fix error --- yb-voyager/test/containers/yugabytedb_container.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yb-voyager/test/containers/yugabytedb_container.go b/yb-voyager/test/containers/yugabytedb_container.go index 0538a1f6d7..c2900d12b9 100644 --- a/yb-voyager/test/containers/yugabytedb_container.go +++ b/yb-voyager/test/containers/yugabytedb_container.go @@ -141,7 +141,7 @@ func (yb *YugabyteDBContainer) ExecuteSqls(sqls ...string) { time.Sleep(2 * time.Second) } if err != nil { - utils.ErrExit("failed to execute sql: %v", err) + utils.ErrExit("failed to execute sql '%s': %w", sql, err) } } } From 02a4f1faed09c910d3341a21787845f0bf9480ec Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 12:39:10 +0530 Subject: [PATCH 53/87] base --- yb-voyager/cmd/importDataFileTaskPicker.go | 67 ++++++++++++++++++++++ 1 file changed, 67 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index c8620e9795..6cc94b1c57 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -17,6 +17,9 @@ package cmd import ( "fmt" + + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" ) type FileTaskPicker interface { @@ -81,3 +84,67 @@ func (s *SequentialTaskPicker) MarkTaskAsDone(task *ImportFileTask) error { func (s *SequentialTaskPicker) HasMoreTasks() bool { return len(s.pendingTasks) > 0 } + +/* +Pick any table at random, but tables have probabilities. +All sharded tables have equal probabilities. +The probabilities of all colocated tables sum up to the probability of a single sharded table. + + In essence, since all colocated tables are in a single tablet, they are all considered + to represent one single table from the perspective of picking. + + If there are 4 colocated tables and 4 sharded tables, + the probabilities for each of the sharded tables will be 1/5 (4 sharded + 1 for colocated) = 0.2. + The probabilities of each of the colocated tables will be 0.2/4 = 0.05. + Therefore, (0.05 x 4) + (0.2 x 4) = 1 + +At any given time, only X distinct tables can be IN-PROGRESS. If X=4, after picking four distinct tables, + + we will not pick a new 5th table. Only when one of the four is completely imported, + we can go on to pick a different table. This is just to make it slightly easier from a + status reporting/debugging perspective. + X >= N (no. of nodes in the cluster). This will lead to a better chance of achieving even load on the cluster. +*/ +type ColocatedAwareRandomTaskPicker struct { + pendingTasks []*ImportFileTask + doneTasks []*ImportFileTask + inProgressTasks []*ImportFileTask + + maxTasksInProgress int + + tableWisePendingTasks *utils.StructMap[sqlname.NameTuple, []*ImportFileTask] + tableType *utils.StructMap[sqlname.NameTuple, string] //colocated or sharded +} + +type YbTargetDBColocatedChecker interface { + IsDBColocated() (bool, error) + IsTableColocated(tableName sqlname.NameTuple) (bool, error) +} + +func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFileTask, state *ImportDataState) (*ColocatedAwareRandomTaskPicker, error) { + var pendingTasks []*ImportFileTask + var doneTasks []*ImportFileTask + var inProgressTasks []*ImportFileTask + for _, task := range tasks { + taskStatus, err := state.GetFileImportState(task.FilePath, task.TableNameTup) + if err != nil { + return nil, fmt.Errorf("getting file import state for tasl: %v: %w", task, err) + } + switch taskStatus { + case FILE_IMPORT_COMPLETED: + doneTasks = append(doneTasks, task) + case FILE_IMPORT_IN_PROGRESS: + inProgressTasks = append(inProgressTasks, task) + case FILE_IMPORT_NOT_STARTED: + pendingTasks = append(pendingTasks, task) + default: + return nil, fmt.Errorf("unexpected status for task: %v: %v", task, taskStatus) + } + } + return &ColocatedAwareRandomTaskPicker{ + pendingTasks: pendingTasks, + doneTasks: doneTasks, + inProgressTasks: inProgressTasks, + maxTasksInProgress: maxTasksInProgress, + }, nil +} From 785a0f212ac8fda218bff599ae338710b96707e8 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 12:52:46 +0530 Subject: [PATCH 54/87] init picker --- yb-voyager/cmd/importDataFileTaskPicker.go | 60 ++++++++++++++++++---- 1 file changed, 51 insertions(+), 9 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 6cc94b1c57..1d3b999632 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -18,10 +18,16 @@ package cmd import ( "fmt" + "github.com/samber/lo" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" ) +const ( + SHARDED = "sharded" + COLOCATED = "colocated" +) + type FileTaskPicker interface { NextTask() (*ImportFileTask, error) MarkTaskAsDone(task *ImportFileTask) error @@ -106,14 +112,14 @@ At any given time, only X distinct tables can be IN-PROGRESS. If X=4, after pic X >= N (no. of nodes in the cluster). This will lead to a better chance of achieving even load on the cluster. */ type ColocatedAwareRandomTaskPicker struct { - pendingTasks []*ImportFileTask + // pendingTasks []*ImportFileTask doneTasks []*ImportFileTask inProgressTasks []*ImportFileTask maxTasksInProgress int tableWisePendingTasks *utils.StructMap[sqlname.NameTuple, []*ImportFileTask] - tableType *utils.StructMap[sqlname.NameTuple, string] //colocated or sharded + tableTypes *utils.StructMap[sqlname.NameTuple, string] //colocated or sharded } type YbTargetDBColocatedChecker interface { @@ -121,11 +127,37 @@ type YbTargetDBColocatedChecker interface { IsTableColocated(tableName sqlname.NameTuple) (bool, error) } -func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFileTask, state *ImportDataState) (*ColocatedAwareRandomTaskPicker, error) { - var pendingTasks []*ImportFileTask +func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFileTask, state *ImportDataState, yb YbTargetDBColocatedChecker) (*ColocatedAwareRandomTaskPicker, error) { + // var pendingTasks []*ImportFileTask var doneTasks []*ImportFileTask var inProgressTasks []*ImportFileTask + tableWisePendingTasks := utils.NewStructMap[sqlname.NameTuple, []*ImportFileTask]() + tableTypes := utils.NewStructMap[sqlname.NameTuple, string]() + + isDBColocated, err := yb.IsDBColocated() + if err != nil { + return nil, fmt.Errorf("checking if db is colocated: %w", err) + } + for _, task := range tasks { + tableName := task.TableNameTup + + // set tableType if not already set + if _, ok := tableTypes.Get(tableName); !ok { + var tableType string + if !isDBColocated { + tableType = SHARDED + } else { + isColocated, err := yb.IsTableColocated(tableName) + if err != nil { + return nil, fmt.Errorf("checking if table is colocated: table: %v: %w", tableName, err) + } + tableType = lo.Ternary(isColocated, COLOCATED, SHARDED) + } + tableTypes.Put(tableName, tableType) + } + + // put task into right bucket. taskStatus, err := state.GetFileImportState(task.FilePath, task.TableNameTup) if err != nil { return nil, fmt.Errorf("getting file import state for tasl: %v: %w", task, err) @@ -136,15 +168,25 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi case FILE_IMPORT_IN_PROGRESS: inProgressTasks = append(inProgressTasks, task) case FILE_IMPORT_NOT_STARTED: - pendingTasks = append(pendingTasks, task) + // put into the table wise pending tasks. + var tablePendingTasks []*ImportFileTask + var ok bool + tablePendingTasks, ok = tableWisePendingTasks.Get(tableName) + if !ok { + tablePendingTasks = []*ImportFileTask{} + } + tablePendingTasks = append(tablePendingTasks, task) + tableWisePendingTasks.Put(tableName, tablePendingTasks) default: return nil, fmt.Errorf("unexpected status for task: %v: %v", task, taskStatus) } } + return &ColocatedAwareRandomTaskPicker{ - pendingTasks: pendingTasks, - doneTasks: doneTasks, - inProgressTasks: inProgressTasks, - maxTasksInProgress: maxTasksInProgress, + doneTasks: doneTasks, + inProgressTasks: inProgressTasks, + maxTasksInProgress: maxTasksInProgress, + tableWisePendingTasks: tableWisePendingTasks, + tableTypes: tableTypes, }, nil } From 0e02acf0ad3a52cd2bf5b1ca75ddde10eeb2d53f Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 12:59:01 +0530 Subject: [PATCH 55/87] HasMoreTasks, MarkTaskAsDone --- yb-voyager/cmd/importDataFileTaskPicker.go | 28 ++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 1d3b999632..8c8b1433b3 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -190,3 +190,31 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi tableTypes: tableTypes, }, nil } + +func (c *ColocatedAwareRandomTaskPicker) MarkTaskAsDone(task *ImportFileTask) error { + for i, t := range c.inProgressTasks { + if t.ID == task.ID { + c.inProgressTasks = append(c.inProgressTasks[:i], c.inProgressTasks[i+1:]...) + c.doneTasks = append(c.doneTasks, task) + return nil + } + } + return fmt.Errorf("task [%v] not found in inProgressTasks: %v", task, c.inProgressTasks) +} + +func (c *ColocatedAwareRandomTaskPicker) HasMoreTasks() bool { + if len(c.inProgressTasks) > 0 { + return true + } + + pendingTasks := false + c.tableWisePendingTasks.IterKV(func(tableName sqlname.NameTuple, tasks []*ImportFileTask) (bool, error) { + if len(tasks) > 0 { + pendingTasks = true + return false, nil + } + return true, nil + }) + + return pendingTasks +} From 56df88d1c426a367dc4974b9636f7cb70c198100 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 13:54:34 +0530 Subject: [PATCH 56/87] NextTask WIP --- yb-voyager/cmd/importDataFileTaskPicker.go | 35 ++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 8c8b1433b3..78e5877bb4 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -120,6 +120,7 @@ type ColocatedAwareRandomTaskPicker struct { tableWisePendingTasks *utils.StructMap[sqlname.NameTuple, []*ImportFileTask] tableTypes *utils.StructMap[sqlname.NameTuple, string] //colocated or sharded + tableProbabilities *utils.StructMap[sqlname.NameTuple, float64] } type YbTargetDBColocatedChecker interface { @@ -191,6 +192,40 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi }, nil } +func (c *ColocatedAwareRandomTaskPicker) NextTask() (*ImportFileTask, error) { + if !c.HasMoreTasks() { + return nil, fmt.Errorf("no more tasks") + } + + // compute probabilities if not already + if c.tableProbabilities == nil { + c.computeProbabilities() + } + +} + +func (c *ColocatedAwareRandomTaskPicker) computeProbabilities() error { + tableNames := make([]sqlname.NameTuple, 0, len(c.tableWisePendingTasks.Keys())) + c.tableWisePendingTasks.IterKV(func(k sqlname.NameTuple, v []*ImportFileTask) (bool, error) { + tableNames = append(tableNames, k) + return true, nil + }) + + totalCount := len(tableNames) + colocatedCount := 0 + for _, tableName := range tableNames { + tableType, ok := c.tableTypes.Get(tableName) + if !ok { + return fmt.Errorf("table type not found for table: %v", tableName) + } + + if tableType == COLOCATED { + colocatedCount++ + } + } + +} + func (c *ColocatedAwareRandomTaskPicker) MarkTaskAsDone(task *ImportFileTask) error { for i, t := range c.inProgressTasks { if t.ID == task.ID { From 126311fa242c8b1b8b04a062fb5fe213188417b7 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 14:31:14 +0530 Subject: [PATCH 57/87] NextTask --- yb-voyager/cmd/importDataFileTaskPicker.go | 86 +++++++++++++++++++--- yb-voyager/go.mod | 1 + yb-voyager/go.sum | 2 + 3 files changed, 78 insertions(+), 11 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 78e5877bb4..ce365fbb4b 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -18,9 +18,11 @@ package cmd import ( "fmt" + "github.com/mroth/weightedrand/v2" "github.com/samber/lo" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" + "golang.org/x/exp/rand" ) const ( @@ -113,14 +115,14 @@ At any given time, only X distinct tables can be IN-PROGRESS. If X=4, after pic */ type ColocatedAwareRandomTaskPicker struct { // pendingTasks []*ImportFileTask - doneTasks []*ImportFileTask - inProgressTasks []*ImportFileTask + doneTasks []*ImportFileTask + inProgressTasks []*ImportFileTask + tableWisePendingTasks *utils.StructMap[sqlname.NameTuple, []*ImportFileTask] + maxTasksInProgress int - maxTasksInProgress int + tableTypes *utils.StructMap[sqlname.NameTuple, string] //colocated or sharded - tableWisePendingTasks *utils.StructMap[sqlname.NameTuple, []*ImportFileTask] - tableTypes *utils.StructMap[sqlname.NameTuple, string] //colocated or sharded - tableProbabilities *utils.StructMap[sqlname.NameTuple, float64] + tableChooser *weightedrand.Chooser[sqlname.NameTuple, int] } type YbTargetDBColocatedChecker interface { @@ -197,21 +199,64 @@ func (c *ColocatedAwareRandomTaskPicker) NextTask() (*ImportFileTask, error) { return nil, fmt.Errorf("no more tasks") } - // compute probabilities if not already - if c.tableProbabilities == nil { - c.computeProbabilities() + if c.tableChooser == nil { + c.initializeChooser() + } + + // if we have already picked maxTasksInProgress tasks, pick a task from inProgressTasks + if len(c.inProgressTasks) == c.maxTasksInProgress { + return c.PickTaskFromInProgressTasks() + } + + // pick a new task from pending tasks + return c.PickTaskFromPendingTasks() +} + +func (c *ColocatedAwareRandomTaskPicker) PickTaskFromInProgressTasks() (*ImportFileTask, error) { + if len(c.inProgressTasks) == 0 { + return nil, fmt.Errorf("no tasks in progress") } + // pick a random task from inProgressTasks + taskIndex := rand.Intn(len(c.inProgressTasks)) + return c.inProgressTasks[taskIndex], nil +} + +func (c *ColocatedAwareRandomTaskPicker) PickTaskFromPendingTasks() (*ImportFileTask, error) { + if len(c.tableWisePendingTasks.Keys()) == 0 { + return nil, fmt.Errorf("no pending tasks to pick from") + } + tablePick := c.tableChooser.Pick() + tablePendingTasks, ok := c.tableWisePendingTasks.Get(tablePick) + if !ok { + return nil, fmt.Errorf("no pending tasks for table picked: %s: %v", tablePick, c.tableWisePendingTasks) + } + + pickedTask := tablePendingTasks[0] + tablePendingTasks = tablePendingTasks[1:] + + if len(tablePendingTasks) == 0 { + c.tableWisePendingTasks.Delete(tablePick) + + // reinitialize chooser because we have removed a table from the pending list, so weights will change. + err := c.initializeChooser() + if err != nil { + return nil, fmt.Errorf("re-initializing chooser after picking task: %v: %w", pickedTask, err) + } + } else { + c.tableWisePendingTasks.Put(tablePick, tablePendingTasks) + } + c.inProgressTasks = append(c.inProgressTasks, pickedTask) + return pickedTask, nil } -func (c *ColocatedAwareRandomTaskPicker) computeProbabilities() error { +func (c *ColocatedAwareRandomTaskPicker) initializeChooser() error { tableNames := make([]sqlname.NameTuple, 0, len(c.tableWisePendingTasks.Keys())) c.tableWisePendingTasks.IterKV(func(k sqlname.NameTuple, v []*ImportFileTask) (bool, error) { tableNames = append(tableNames, k) return true, nil }) - totalCount := len(tableNames) colocatedCount := 0 for _, tableName := range tableNames { tableType, ok := c.tableTypes.Get(tableName) @@ -224,6 +269,25 @@ func (c *ColocatedAwareRandomTaskPicker) computeProbabilities() error { } } + choices := []weightedrand.Choice[sqlname.NameTuple, int]{} + for _, tableName := range tableNames { + tableType, ok := c.tableTypes.Get(tableName) + if !ok { + return fmt.Errorf("table type not found for table: %v", tableName) + } + if tableType == COLOCATED { + choices = append(choices, weightedrand.NewChoice(tableName, 1)) + } else { + choices = append(choices, weightedrand.NewChoice(tableName, colocatedCount)) + } + + var err error + c.tableChooser, err = weightedrand.NewChooser(choices...) + if err != nil { + return fmt.Errorf("creating chooser: %w", err) + } + } + return nil } func (c *ColocatedAwareRandomTaskPicker) MarkTaskAsDone(task *ImportFileTask) error { diff --git a/yb-voyager/go.mod b/yb-voyager/go.mod index 228330bfa1..f4364dbb78 100644 --- a/yb-voyager/go.mod +++ b/yb-voyager/go.mod @@ -74,6 +74,7 @@ require ( github.com/moby/sys/user v0.1.0 // indirect github.com/moby/term v0.5.0 // indirect github.com/morikuni/aec v1.0.0 // indirect + github.com/mroth/weightedrand/v2 v2.1.0 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.1.0 // indirect github.com/pkg/errors v0.9.1 // indirect diff --git a/yb-voyager/go.sum b/yb-voyager/go.sum index 61e5485128..10e982647f 100644 --- a/yb-voyager/go.sum +++ b/yb-voyager/go.sum @@ -1603,6 +1603,8 @@ github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJ github.com/montanaflynn/stats v0.6.6/go.mod h1:etXPPgVO6n31NxCd9KQUMvCM+ve0ruNzt6R8Bnaayow= github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= +github.com/mroth/weightedrand/v2 v2.1.0 h1:o1ascnB1CIVzsqlfArQQjeMy1U0NcIbBO5rfd5E/OeU= +github.com/mroth/weightedrand/v2 v2.1.0/go.mod h1:f2faGsfOGOwc1p94wzHKKZyTpcJUW7OJ/9U4yfiNAOU= github.com/mrunalp/fileutils v0.5.0/go.mod h1:M1WthSahJixYnrXQl/DFQuteStB1weuxD2QJNHXfbSQ= github.com/munnerz/goautoneg v0.0.0-20120707110453-a547fc61f48d/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= From fceb7bc8fd0227326b8de52bffeec3d4b9d56f83 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 16:27:26 +0530 Subject: [PATCH 58/87] basic test and some fixes --- yb-voyager/cmd/importDataFileTaskPicker.go | 11 +- .../cmd/importDataFileTaskPicker_test.go | 116 ++++++++++++++++++ 2 files changed, 125 insertions(+), 2 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index ce365fbb4b..7e5df35785 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -208,6 +208,11 @@ func (c *ColocatedAwareRandomTaskPicker) NextTask() (*ImportFileTask, error) { return c.PickTaskFromInProgressTasks() } + // if we have less than maxTasksInProgress tasks in progress, but no pending tasks, pick a task from inProgressTasks + if len(c.inProgressTasks) < c.maxTasksInProgress && len(c.tableWisePendingTasks.Keys()) == 0 { + return c.PickTaskFromInProgressTasks() + } + // pick a new task from pending tasks return c.PickTaskFromPendingTasks() } @@ -268,6 +273,8 @@ func (c *ColocatedAwareRandomTaskPicker) initializeChooser() error { colocatedCount++ } } + colocatedWeight := 1 + shardedWeight := lo.Ternary(colocatedCount == 0, 1, colocatedCount) choices := []weightedrand.Choice[sqlname.NameTuple, int]{} for _, tableName := range tableNames { @@ -276,9 +283,9 @@ func (c *ColocatedAwareRandomTaskPicker) initializeChooser() error { return fmt.Errorf("table type not found for table: %v", tableName) } if tableType == COLOCATED { - choices = append(choices, weightedrand.NewChoice(tableName, 1)) + choices = append(choices, weightedrand.NewChoice(tableName, colocatedWeight)) } else { - choices = append(choices, weightedrand.NewChoice(tableName, colocatedCount)) + choices = append(choices, weightedrand.NewChoice(tableName, shardedWeight)) } var err error diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 5d2826012f..93276df994 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -21,9 +21,28 @@ import ( "testing" "github.com/stretchr/testify/assert" + "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" ) +type dummyYb struct { + colocatedTables []sqlname.NameTuple + shardedTables []sqlname.NameTuple +} + +func (d *dummyYb) IsTableColocated(tableName sqlname.NameTuple) (bool, error) { + for _, t := range d.colocatedTables { + if t.Key() == tableName.Key() { + return true, nil + } + } + return false, nil +} + +func (d *dummyYb) IsDBColocated() (bool, error) { + return len(d.colocatedTables) > 0, nil +} + func TestSequentialTaskPickerBasic(t *testing.T) { ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) testutils.FatalIfError(t, err) @@ -114,3 +133,100 @@ func TestSequentialTaskPickerMarkTaskDone(t *testing.T) { err = picker.MarkTaskAsDone(task1) assert.Error(t, err) } + +func TestColocatedAwareRandomTaskPickerAdheresMaxInProgress(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask1, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated1", 2) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated2", 2) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + shardedTask1, + colocatedTask1, + colocatedTask2, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + }, + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + }, + } + + picker, err := NewColocatedAwareRandomTaskPicker(2, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // initially because maxInprogressTasks is 2, we will get 2 different tasks + pickedTask1, err := picker.NextTask() + assert.NoError(t, err) + pickedTask2, err := picker.NextTask() + assert.NoError(t, err) + assert.NotEqual(t, pickedTask1, pickedTask2) + + // no matter how many times we call NextTask therefater, + // it should return either pickedTask1 or pickedTask2 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.True(t, task == pickedTask1 || task == pickedTask2) + } + + // mark task1 as done + err = picker.MarkTaskAsDone(pickedTask1) + assert.NoError(t, err) + + // keep picking tasks until we get a task that is not pickedTask2 + var pickedTask3 *ImportFileTask + for { + task, err := picker.NextTask() + if err != nil { + break + } + if task != pickedTask2 { + pickedTask3 = task + break + } + } + + // now, next task should be either pickedTask2 or pickedTask3 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.True(t, task == pickedTask2 || task == pickedTask3) + } + + // mark task3 as done + err = picker.MarkTaskAsDone(pickedTask3) + assert.NoError(t, err) + + // now, next task should be pickedTask2 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.True(t, task == pickedTask2) + } + + // // mark task2 as done + // err = picker.MarkTaskAsDone(pickedTask2) + // assert.NoError(t, err) + + // // now, there should be no more tasks + // assert.False(t, picker.HasMoreTasks()) + // _, err = picker.NextTask() + // assert.Error(t, err) +} From df0a35e3c715ffdbe393ee35a1fcbe41a943448b Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 16:32:02 +0530 Subject: [PATCH 59/87] fix test --- .../cmd/importDataFileTaskPicker_test.go | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 93276df994..2abe3889cf 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -134,7 +134,7 @@ func TestSequentialTaskPickerMarkTaskDone(t *testing.T) { assert.Error(t, err) } -func TestColocatedAwareRandomTaskPickerAdheresMaxInProgress(t *testing.T) { +func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) { ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) testutils.FatalIfError(t, err) @@ -149,7 +149,7 @@ func TestColocatedAwareRandomTaskPickerAdheresMaxInProgress(t *testing.T) { testutils.FatalIfError(t, err) _, colocatedTask1, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated1", 2) testutils.FatalIfError(t, err) - _, colocatedTask2, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated2", 2) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated2", 3) testutils.FatalIfError(t, err) tasks := []*ImportFileTask{ @@ -183,7 +183,7 @@ func TestColocatedAwareRandomTaskPickerAdheresMaxInProgress(t *testing.T) { for i := 0; i < 100; i++ { task, err := picker.NextTask() assert.NoError(t, err) - assert.True(t, task == pickedTask1 || task == pickedTask2) + assert.Truef(t, task == pickedTask1 || task == pickedTask2, "task: %v, pickedTask1: %v, pickedTask2: %v", task, pickedTask1, pickedTask2) } // mark task1 as done @@ -207,7 +207,7 @@ func TestColocatedAwareRandomTaskPickerAdheresMaxInProgress(t *testing.T) { for i := 0; i < 100; i++ { task, err := picker.NextTask() assert.NoError(t, err) - assert.True(t, task == pickedTask2 || task == pickedTask3) + assert.Truef(t, task == pickedTask2 || task == pickedTask3, "task: %v, pickedTask2: %v, pickedTask3: %v", task, pickedTask2, pickedTask3) } // mark task3 as done @@ -218,15 +218,15 @@ func TestColocatedAwareRandomTaskPickerAdheresMaxInProgress(t *testing.T) { for i := 0; i < 100; i++ { task, err := picker.NextTask() assert.NoError(t, err) - assert.True(t, task == pickedTask2) + assert.Truef(t, task == pickedTask2, "task: %v, pickedTask2: %v", task, pickedTask2) } - // // mark task2 as done - // err = picker.MarkTaskAsDone(pickedTask2) - // assert.NoError(t, err) + // mark task2 as done + err = picker.MarkTaskAsDone(pickedTask2) + assert.NoError(t, err) - // // now, there should be no more tasks - // assert.False(t, picker.HasMoreTasks()) - // _, err = picker.NextTask() - // assert.Error(t, err) + // now, there should be no more tasks + assert.False(t, picker.HasMoreTasks()) + _, err = picker.NextTask() + assert.Error(t, err) } From 64bf875764deef770b19eddc5fd1579b5166fb7e Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 16:42:11 +0530 Subject: [PATCH 60/87] comments --- yb-voyager/cmd/importDataFileTaskPicker_test.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 2abe3889cf..cf195a9c0b 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -230,3 +230,14 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) _, err = picker.NextTask() assert.Error(t, err) } + +/* +maxTasksInProgressEqualToTotalTasks +maxTasksInProgressLessThanTotalTasks +MixOfColocatedAndShardedTables - ensure all are getting picked. +AllColocatedTables - ensure all are getting picked +AllShardedTables - ensure all are getting picked + +all of the above cases for +multipleTasksPerTable (importDataFileCase) +*/ From 88169c1a1e53073fdbddb92425941e92beb0c3ae Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 16:44:16 +0530 Subject: [PATCH 61/87] test cases --- yb-voyager/cmd/importDataFileTaskPicker_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index cf195a9c0b..ec154e273a 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -233,10 +233,10 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) /* maxTasksInProgressEqualToTotalTasks -maxTasksInProgressLessThanTotalTasks -MixOfColocatedAndShardedTables - ensure all are getting picked. -AllColocatedTables - ensure all are getting picked -AllShardedTables - ensure all are getting picked +maxTasksInProgressGreaterThanTotalTasks +MixOfColocatedAndShardedTables - ensure all are getting picked with proper weights +AllColocatedTables - ensure all are getting picked with proper weights +AllShardedTables - ensure all are getting picked with proper weights all of the above cases for multipleTasksPerTable (importDataFileCase) From 91ba7057cf45ee967d104ad110b7efad2f852b9d Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 17:20:37 +0530 Subject: [PATCH 62/87] comment --- yb-voyager/cmd/importDataFileTaskPicker_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index ec154e273a..0d46308297 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -232,6 +232,7 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) } /* +singleTask maxTasksInProgressEqualToTotalTasks maxTasksInProgressGreaterThanTotalTasks MixOfColocatedAndShardedTables - ensure all are getting picked with proper weights From 4183f0f6426ead3e21eaa152098003388195c9ee Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 17:26:01 +0530 Subject: [PATCH 63/87] test idea --- yb-voyager/cmd/importDataFileTaskPicker_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 0d46308297..a202f8c584 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -238,6 +238,10 @@ maxTasksInProgressGreaterThanTotalTasks MixOfColocatedAndShardedTables - ensure all are getting picked with proper weights AllColocatedTables - ensure all are getting picked with proper weights AllShardedTables - ensure all are getting picked with proper weights +dynamicTasks - create 100s of tasks randmoly colocated/shared, + get next tasks in a large loop, + mark them as done occasionally, + ensure all are getting picked at the end all of the above cases for multipleTasksPerTable (importDataFileCase) From bdec1d776379cb1200c4964f9fd46d875b6fd51f Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 17:27:28 +0530 Subject: [PATCH 64/87] test idea --- yb-voyager/cmd/importDataFileTaskPicker_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index a202f8c584..1de60bc34a 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -241,7 +241,8 @@ AllShardedTables - ensure all are getting picked with proper weights dynamicTasks - create 100s of tasks randmoly colocated/shared, get next tasks in a large loop, mark them as done occasionally, - ensure all are getting picked at the end + ensure all are getting picked at least once. + ensure tasks getting picked remain constant between marking them as done. and adhere to max. all of the above cases for multipleTasksPerTable (importDataFileCase) From 585d1a3791353adf90dde82c96f9300ae2981158 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 29 Jan 2025 18:26:34 +0530 Subject: [PATCH 65/87] test idea --- yb-voyager/cmd/importDataFileTaskPicker_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 1de60bc34a..8961cf3488 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -236,8 +236,11 @@ singleTask maxTasksInProgressEqualToTotalTasks maxTasksInProgressGreaterThanTotalTasks MixOfColocatedAndShardedTables - ensure all are getting picked with proper weights + test that weights change when tasks are marked as done AllColocatedTables - ensure all are getting picked with proper weights + test that weights change when tasks are marked as done AllShardedTables - ensure all are getting picked with proper weights + test that weights change when tasks are marked as done dynamicTasks - create 100s of tasks randmoly colocated/shared, get next tasks in a large loop, mark them as done occasionally, From 70d345a5f02b8f2e7f696916a069e02f1c608b17 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 30 Jan 2025 14:32:11 +0530 Subject: [PATCH 66/87] tests colocated, sharded --- .../cmd/importDataFileTaskPicker_test.go | 310 ++++++++++++++++++ 1 file changed, 310 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 8961cf3488..43e743e769 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -17,6 +17,7 @@ package cmd import ( "fmt" + "math" "os" "testing" @@ -231,6 +232,315 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) assert.Error(t, err) } +func TestColocatedAwareRandomTaskPickerSingleTask(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + shardedTask1, + } + dummyYb := &dummyYb{ + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + }, + } + + picker, err := NewColocatedAwareRandomTaskPicker(2, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + pickedTask1, err := picker.NextTask() + assert.NoError(t, err) + + // no matter how many times we call NextTask therefater, + // it should return either pickedTask1 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == pickedTask1, "task: %v, pickedTask1: %v", task, pickedTask1) + } + + // mark task1 as done + err = picker.MarkTaskAsDone(pickedTask1) + assert.NoError(t, err) + + // now, there should be no more tasks + assert.False(t, picker.HasMoreTasks()) + _, err = picker.NextTask() + assert.Error(t, err) +} + +func TestColocatedAwareRandomTaskPickerTasksEqualToMaxTasksInProgress(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask1, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated1", 2) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated2", 3) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + shardedTask1, + colocatedTask1, + colocatedTask2, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + }, + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + }, + } + + // 3 tasks, 3 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(3, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask therefater, + // it should return one of the tasks + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == shardedTask1 || task == colocatedTask1 || task == colocatedTask2, "task: %v, expected tasks = %v", task, tasks) + } + + // mark task1 as done + err = picker.MarkTaskAsDone(shardedTask1) + assert.NoError(t, err) + + // now, next task should be either colocatedTask1 or colocatedTask2 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == colocatedTask1 || task == colocatedTask2, "task: %v, colocatedTask1: %v, colocatedTask2: %v", task, colocatedTask1, colocatedTask2) + } + + // mark colocatedTask1, colocatedTask2 as done + err = picker.MarkTaskAsDone(colocatedTask1) + assert.NoError(t, err) + err = picker.MarkTaskAsDone(colocatedTask2) + assert.NoError(t, err) + + // now, there should be no more tasks + assert.False(t, picker.HasMoreTasks()) + _, err = picker.NextTask() + assert.Error(t, err) +} + +func TestColocatedAwareRandomTaskPickerTasksLessThanMaxTasksInProgress(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask1, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated1", 2) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated2", 3) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + shardedTask1, + colocatedTask1, + colocatedTask2, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + }, + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + }, + } + + // 3 tasks, 10 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask therefater, + // it should return one of the tasks + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == shardedTask1 || task == colocatedTask1 || task == colocatedTask2, "task: %v, expected tasks = %v", task, tasks) + } + + // mark task1 as done + err = picker.MarkTaskAsDone(shardedTask1) + assert.NoError(t, err) + + // now, next task should be either colocatedTask1 or colocatedTask2 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == colocatedTask1 || task == colocatedTask2, "task: %v, colocatedTask1: %v, colocatedTask2: %v", task, colocatedTask1, colocatedTask2) + } + + // mark colocatedTask1, colocatedTask2 as done + err = picker.MarkTaskAsDone(colocatedTask1) + assert.NoError(t, err) + err = picker.MarkTaskAsDone(colocatedTask2) + assert.NoError(t, err) + + // now, there should be no more tasks + assert.False(t, picker.HasMoreTasks()) + _, err = picker.NextTask() + assert.Error(t, err) +} + +func TestColocatedAwareRandomTaskPickerAllShardedTasks(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, shardedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.sharded2", 2) + testutils.FatalIfError(t, err) + _, shardedTask3, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.sharded3", 3) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + shardedTask1, + shardedTask2, + shardedTask3, + } + dummyYb := &dummyYb{ + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + shardedTask2.TableNameTup, + shardedTask3.TableNameTup, + }, + } + + // 3 tasks, 10 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask therefater, + // it should return one of the tasks with equal probability. + taskCounter := map[*ImportFileTask]int{} + for i := 0; i < 100000; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + taskCounter[task]++ + } + + totalTasks := len(tasks) + assert.Equal(t, totalTasks, len(taskCounter)) + tc := 0 + for _, v := range taskCounter { + tc += v + } + + expectedCountForEachTask := tc / totalTasks + for _, v := range taskCounter { + diff := math.Abs(float64(v - expectedCountForEachTask)) + diffPct := diff / float64(expectedCountForEachTask) * 100 + // pct difference from expected count should be less than 5% + assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + } +} + +func TestColocatedAwareRandomTaskPickerAllColocatedTasks(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, colocatedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.colocated1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated2", 2) + testutils.FatalIfError(t, err) + _, colocatedTask3, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated3", 3) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + colocatedTask1, + colocatedTask2, + colocatedTask3, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + colocatedTask3.TableNameTup, + }, + } + + // 3 tasks, 10 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask therefater, + // it should return one of the tasks with equal probability. + taskCounter := map[*ImportFileTask]int{} + for i := 0; i < 100000; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + taskCounter[task]++ + } + + totalTasks := len(tasks) + assert.Equal(t, totalTasks, len(taskCounter)) + tc := 0 + for _, v := range taskCounter { + tc += v + } + + expectedCountForEachTask := tc / totalTasks + for _, v := range taskCounter { + fmt.Printf("count: %v, expectedCountForEachTask: %v\n", v, expectedCountForEachTask) + diff := math.Abs(float64(v - expectedCountForEachTask)) + diffPct := diff / float64(expectedCountForEachTask) * 100 + // pct difference from expected count should be less than 5% + assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + } +} + /* singleTask maxTasksInProgressEqualToTotalTasks From 9f3e028f560c143b0eb9e50be19614832434c0ed Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 30 Jan 2025 14:43:51 +0530 Subject: [PATCH 67/87] bug fix --- yb-voyager/cmd/importDataFileTaskPicker.go | 21 +++--- .../cmd/importDataFileTaskPicker_test.go | 72 +++++++++++++++++++ 2 files changed, 85 insertions(+), 8 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 7e5df35785..fe43fa53e1 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -244,9 +244,11 @@ func (c *ColocatedAwareRandomTaskPicker) PickTaskFromPendingTasks() (*ImportFile c.tableWisePendingTasks.Delete(tablePick) // reinitialize chooser because we have removed a table from the pending list, so weights will change. - err := c.initializeChooser() - if err != nil { - return nil, fmt.Errorf("re-initializing chooser after picking task: %v: %w", pickedTask, err) + if len(c.tableWisePendingTasks.Keys()) > 0 { + err := c.initializeChooser() + if err != nil { + return nil, fmt.Errorf("re-initializing chooser after picking task: %v: %w", pickedTask, err) + } } } else { c.tableWisePendingTasks.Put(tablePick, tablePendingTasks) @@ -256,6 +258,9 @@ func (c *ColocatedAwareRandomTaskPicker) PickTaskFromPendingTasks() (*ImportFile } func (c *ColocatedAwareRandomTaskPicker) initializeChooser() error { + if len(c.tableWisePendingTasks.Keys()) == 0 { + return fmt.Errorf("no pending tasks to initialize chooser") + } tableNames := make([]sqlname.NameTuple, 0, len(c.tableWisePendingTasks.Keys())) c.tableWisePendingTasks.IterKV(func(k sqlname.NameTuple, v []*ImportFileTask) (bool, error) { tableNames = append(tableNames, k) @@ -288,11 +293,11 @@ func (c *ColocatedAwareRandomTaskPicker) initializeChooser() error { choices = append(choices, weightedrand.NewChoice(tableName, shardedWeight)) } - var err error - c.tableChooser, err = weightedrand.NewChooser(choices...) - if err != nil { - return fmt.Errorf("creating chooser: %w", err) - } + } + var err error + c.tableChooser, err = weightedrand.NewChooser(choices...) + if err != nil { + return fmt.Errorf("creating chooser: %w", err) } return nil } diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 43e743e769..d39dd3fd8d 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -541,6 +541,78 @@ func TestColocatedAwareRandomTaskPickerAllColocatedTasks(t *testing.T) { } } +func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasks(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, colocatedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.colocated1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated2", 2) + testutils.FatalIfError(t, err) + _, colocatedTask3, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated3", 3) + testutils.FatalIfError(t, err) + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, shardedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.sharded2", 2) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + colocatedTask1, + colocatedTask2, + colocatedTask3, + shardedTask1, + shardedTask2, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + colocatedTask3.TableNameTup, + }, + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + shardedTask2.TableNameTup, + }, + } + + // 5 tasks, 10 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask therefater, + // it should return one of the tasks with equal probability. + taskCounter := map[*ImportFileTask]int{} + for i := 0; i < 100000; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + taskCounter[task]++ + } + + totalTasks := len(tasks) + assert.Equal(t, totalTasks, len(taskCounter)) + tc := 0 + for _, v := range taskCounter { + tc += v + } + + expectedCountForEachTask := tc / totalTasks + for task, v := range taskCounter { + fmt.Printf("task: %v, count: %v, expectedCountForEachTask: %v\n", task, v, expectedCountForEachTask) + diff := math.Abs(float64(v - expectedCountForEachTask)) + diffPct := diff / float64(expectedCountForEachTask) * 100 + // pct difference from expected count should be less than 5% + assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + } +} + /* singleTask maxTasksInProgressEqualToTotalTasks From 5ebf60b416e8d15df11b3bc93553626c7125cf80 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 30 Jan 2025 14:57:04 +0530 Subject: [PATCH 68/87] comment --- yb-voyager/cmd/importDataFileTaskPicker_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index d39dd3fd8d..69c7152a51 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -588,7 +588,9 @@ func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasks(t *testing.T) { assert.True(t, picker.HasMoreTasks()) // no matter how many times we call NextTask therefater, - // it should return one of the tasks with equal probability. + // it should return one of the tasks with equal probability, + // because they will get picked with their own probability, but once they are picked, + // and are placed in inProgressTasks, their probability will all be equal. taskCounter := map[*ImportFileTask]int{} for i := 0; i < 100000; i++ { task, err := picker.NextTask() From 8e9be5b461fdf949fa7f1ad442b6d9c1f8e220e6 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 5 Feb 2025 15:15:53 +0530 Subject: [PATCH 69/87] empty From f2023a71611c80ccc6b81cb1da372c29aff98be2 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 5 Feb 2025 15:55:01 +0530 Subject: [PATCH 70/87] tests --- yb-voyager/cmd/importDataFileTaskPicker.go | 2 +- .../cmd/importDataFileTaskPicker_test.go | 328 +++++++++++++++--- 2 files changed, 286 insertions(+), 44 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index fe43fa53e1..226eeac54e 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -111,7 +111,7 @@ At any given time, only X distinct tables can be IN-PROGRESS. If X=4, after pic we will not pick a new 5th table. Only when one of the four is completely imported, we can go on to pick a different table. This is just to make it slightly easier from a status reporting/debugging perspective. - X >= N (no. of nodes in the cluster). This will lead to a better chance of achieving even load on the cluster. + During this time, each of the in-progress tables will be picked with equal probability. */ type ColocatedAwareRandomTaskPicker struct { // pendingTasks []*ImportFileTask diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 69c7152a51..6ccdd37b1a 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -455,28 +455,103 @@ func TestColocatedAwareRandomTaskPickerAllShardedTasks(t *testing.T) { assert.True(t, picker.HasMoreTasks()) // no matter how many times we call NextTask therefater, - // it should return one of the tasks with equal probability. - taskCounter := map[*ImportFileTask]int{} - for i := 0; i < 100000; i++ { + // it should return one of the tasks + for i := 0; i < 100; i++ { task, err := picker.NextTask() assert.NoError(t, err) - taskCounter[task]++ + assert.Truef(t, task == shardedTask1 || task == shardedTask2 || task == shardedTask3, "task: %v, expected tasks = %v", task, tasks) + } + + // taskCounter := map[*ImportFileTask]int{} + // for i := 0; i < 100000; i++ { + // task, err := picker.NextTask() + // assert.NoError(t, err) + // taskCounter[task]++ + // } + + // totalTasks := len(tasks) + // assert.Equal(t, totalTasks, len(taskCounter)) + // tc := 0 + // for _, v := range taskCounter { + // tc += v + // } + + // expectedCountForEachTask := tc / totalTasks + // for _, v := range taskCounter { + // diff := math.Abs(float64(v - expectedCountForEachTask)) + // diffPct := diff / float64(expectedCountForEachTask) * 100 + // // pct difference from expected count should be less than 5% + // assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + // } +} + +func TestColocatedAwareRandomTaskPickerAllShardedTasksChooser(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) } - totalTasks := len(tasks) - assert.Equal(t, totalTasks, len(taskCounter)) - tc := 0 - for _, v := range taskCounter { - tc += v + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, shardedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.sharded2", 2) + testutils.FatalIfError(t, err) + _, shardedTask3, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.sharded3", 3) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + shardedTask1, + shardedTask2, + shardedTask3, } + dummyYb := &dummyYb{ + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + shardedTask2.TableNameTup, + shardedTask3.TableNameTup, + }, + } + tableNameTuples := []sqlname.NameTuple{ + shardedTask1.TableNameTup, + shardedTask2.TableNameTup, + shardedTask3.TableNameTup, + } + + // 3 tasks, 10 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) - expectedCountForEachTask := tc / totalTasks - for _, v := range taskCounter { - diff := math.Abs(float64(v - expectedCountForEachTask)) - diffPct := diff / float64(expectedCountForEachTask) * 100 + // all sharded tables should have equal probability of being picked + // table-wise pick counter + picker.initializeChooser() + pickCounter := map[sqlname.NameTuple]int{} + for i := 0; i < 100000; i++ { + tableName := picker.tableChooser.Pick() + assert.Contains(t, tableNameTuples, tableName) + pickCounter[tableName]++ + } + totalTables := len(tableNameTuples) + totalTablePicks := len(pickCounter) + assert.Equal(t, totalTables, totalTablePicks) + + // each table should have been picked almost equal number of times + totalPicks := 0 + for _, v := range pickCounter { + totalPicks += v + } + expectedCountForEachTable := totalPicks / totalTables + for _, v := range pickCounter { + diff := math.Abs(float64(v - expectedCountForEachTable)) + diffPct := float64(diff) / float64(expectedCountForEachTable) * 100 // pct difference from expected count should be less than 5% assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) } + } func TestColocatedAwareRandomTaskPickerAllColocatedTasks(t *testing.T) { @@ -516,26 +591,76 @@ func TestColocatedAwareRandomTaskPickerAllColocatedTasks(t *testing.T) { assert.True(t, picker.HasMoreTasks()) // no matter how many times we call NextTask therefater, - // it should return one of the tasks with equal probability. - taskCounter := map[*ImportFileTask]int{} - for i := 0; i < 100000; i++ { + // it should return one of the tasks + for i := 0; i < 100; i++ { task, err := picker.NextTask() assert.NoError(t, err) - taskCounter[task]++ + assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3, "task: %v, expected tasks = %v", task, tasks) + } +} + +func TestColocatedAwareRandomTaskPickerAllColocatedTasksChooser(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) } - totalTasks := len(tasks) - assert.Equal(t, totalTasks, len(taskCounter)) - tc := 0 - for _, v := range taskCounter { - tc += v + _, colocatedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.colocated1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated2", 2) + testutils.FatalIfError(t, err) + _, colocatedTask3, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated3", 3) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + colocatedTask1, + colocatedTask2, + colocatedTask3, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + colocatedTask3.TableNameTup, + }, + } + tableNameTuples := []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + colocatedTask3.TableNameTup, } - expectedCountForEachTask := tc / totalTasks - for _, v := range taskCounter { - fmt.Printf("count: %v, expectedCountForEachTask: %v\n", v, expectedCountForEachTask) - diff := math.Abs(float64(v - expectedCountForEachTask)) - diffPct := diff / float64(expectedCountForEachTask) * 100 + // 3 tasks, 10 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // all colocated tables should have equal probability of being picked + picker.initializeChooser() + pickCounter := map[sqlname.NameTuple]int{} + for i := 0; i < 100000; i++ { + tableName := picker.tableChooser.Pick() + assert.Contains(t, tableNameTuples, tableName) + pickCounter[tableName]++ + } + totalTables := len(tableNameTuples) + totalTablePicks := len(pickCounter) + assert.Equal(t, totalTables, totalTablePicks) + + // each table should have been picked almost equal number of times + totalPicks := 0 + for _, v := range pickCounter { + totalPicks += v + } + expectedCountForEachTable := totalPicks / totalTables + for _, v := range pickCounter { + diff := math.Abs(float64(v - expectedCountForEachTable)) + diffPct := float64(diff) / float64(expectedCountForEachTable) * 100 // pct difference from expected count should be less than 5% assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) } @@ -588,28 +713,145 @@ func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasks(t *testing.T) { assert.True(t, picker.HasMoreTasks()) // no matter how many times we call NextTask therefater, - // it should return one of the tasks with equal probability, - // because they will get picked with their own probability, but once they are picked, - // and are placed in inProgressTasks, their probability will all be equal. - taskCounter := map[*ImportFileTask]int{} - for i := 0; i < 100000; i++ { + // it should return one of the tasks + for i := 0; i < 100; i++ { task, err := picker.NextTask() assert.NoError(t, err) - taskCounter[task]++ + assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3 || task == shardedTask1 || task == shardedTask2, "task: %v, expected tasks = %v", task, tasks) + } + + // // no matter how many times we call NextTask therefater, + // // it should return one of the tasks with equal probability, + // // because they will get picked with their own probability, but once they are picked, + // // and are placed in inProgressTasks, their probability will all be equal. + // taskCounter := map[*ImportFileTask]int{} + // for i := 0; i < 100000; i++ { + // task, err := picker.NextTask() + // assert.NoError(t, err) + // taskCounter[task]++ + // } + + // totalTasks := len(tasks) + // assert.Equal(t, totalTasks, len(taskCounter)) + // tc := 0 + // for _, v := range taskCounter { + // tc += v + // } + + // expectedCountForEachTask := tc / totalTasks + // for task, v := range taskCounter { + // fmt.Printf("task: %v, count: %v, expectedCountForEachTask: %v\n", task, v, expectedCountForEachTask) + // diff := math.Abs(float64(v - expectedCountForEachTask)) + // diffPct := diff / float64(expectedCountForEachTask) * 100 + // // pct difference from expected count should be less than 5% + // assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + // } +} + +func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasksChooser(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, colocatedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.colocated1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated2", 2) + testutils.FatalIfError(t, err) + _, colocatedTask3, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated3", 3) + testutils.FatalIfError(t, err) + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, shardedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.sharded2", 2) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + colocatedTask1, + colocatedTask2, + colocatedTask3, + shardedTask1, + shardedTask2, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + colocatedTask3.TableNameTup, + }, + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + shardedTask2.TableNameTup, + }, } + tableNameTuples := append(dummyYb.colocatedTables, dummyYb.shardedTables...) - totalTasks := len(tasks) - assert.Equal(t, totalTasks, len(taskCounter)) - tc := 0 - for _, v := range taskCounter { - tc += v + // 5 tasks, 10 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // all colocated tables should have same probability of being picked + // all sharded tables should have same probability of being picked + // sum of probability of picking any of the colocated tables should be same as probability of picking any of the sharded tables + picker.initializeChooser() + colocatedPickCounter := map[sqlname.NameTuple]int{} + shardedPickCounter := map[sqlname.NameTuple]int{} + + for i := 0; i < 100000; i++ { + tableName := picker.tableChooser.Pick() + assert.Contains(t, tableNameTuples, tableName) + tableType, ok := picker.tableTypes.Get(tableName) + assert.True(t, ok) + + if tableType == COLOCATED { + colocatedPickCounter[tableName]++ + } else { + shardedPickCounter[tableName]++ + } + } + fmt.Printf("colocatedPickCounter: %v\n", colocatedPickCounter) + fmt.Printf("shardedPickCounter: %v\n", shardedPickCounter) + totalColocatedTables := len(dummyYb.colocatedTables) + totalShardedTables := len(dummyYb.shardedTables) + + assert.Equal(t, totalColocatedTables, len(colocatedPickCounter)) + assert.Equal(t, totalShardedTables, len(shardedPickCounter)) + + // assert that all colocated tables have been picked almost equal number of times + totalColocatedPicks := 0 + for _, v := range colocatedPickCounter { + totalColocatedPicks += v + } + expectedCountForEachColocatedTable := totalColocatedPicks / totalColocatedTables + for _, v := range colocatedPickCounter { + diff := math.Abs(float64(v - expectedCountForEachColocatedTable)) + diffPct := float64(diff) / float64(expectedCountForEachColocatedTable) * 100 + // pct difference from expected count should be less than 5% + assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + } + + // assert that all sharded tables have been picked almost equal number of times + totalShardedPicks := 0 + for _, v := range shardedPickCounter { + totalShardedPicks += v + } + expectedCountForEachShardedTable := totalShardedPicks / totalShardedTables + for _, v := range shardedPickCounter { + diff := math.Abs(float64(v - expectedCountForEachShardedTable)) + diffPct := float64(diff) / float64(expectedCountForEachShardedTable) * 100 + // pct difference from expected count should be less than 5% + assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) } - expectedCountForEachTask := tc / totalTasks - for task, v := range taskCounter { - fmt.Printf("task: %v, count: %v, expectedCountForEachTask: %v\n", task, v, expectedCountForEachTask) - diff := math.Abs(float64(v - expectedCountForEachTask)) - diffPct := diff / float64(expectedCountForEachTask) * 100 + // assert that sum of probability of picking any of the colocated tables should be same as probability of picking any of the sharded tables + for _, v := range shardedPickCounter { + diff := math.Abs(float64(v - totalColocatedPicks)) + diffPct := float64(diff) / float64(totalColocatedPicks) * 100 // pct difference from expected count should be less than 5% assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) } From 920583f7036cb52d40dbd5d5440ff8d2d536fbff Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 5 Feb 2025 16:01:31 +0530 Subject: [PATCH 71/87] remove comments --- .../cmd/importDataFileTaskPicker_test.go | 54 ------------------- 1 file changed, 54 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 6ccdd37b1a..a5e5760c5b 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -461,28 +461,6 @@ func TestColocatedAwareRandomTaskPickerAllShardedTasks(t *testing.T) { assert.NoError(t, err) assert.Truef(t, task == shardedTask1 || task == shardedTask2 || task == shardedTask3, "task: %v, expected tasks = %v", task, tasks) } - - // taskCounter := map[*ImportFileTask]int{} - // for i := 0; i < 100000; i++ { - // task, err := picker.NextTask() - // assert.NoError(t, err) - // taskCounter[task]++ - // } - - // totalTasks := len(tasks) - // assert.Equal(t, totalTasks, len(taskCounter)) - // tc := 0 - // for _, v := range taskCounter { - // tc += v - // } - - // expectedCountForEachTask := tc / totalTasks - // for _, v := range taskCounter { - // diff := math.Abs(float64(v - expectedCountForEachTask)) - // diffPct := diff / float64(expectedCountForEachTask) * 100 - // // pct difference from expected count should be less than 5% - // assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) - // } } func TestColocatedAwareRandomTaskPickerAllShardedTasksChooser(t *testing.T) { @@ -719,33 +697,6 @@ func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasks(t *testing.T) { assert.NoError(t, err) assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3 || task == shardedTask1 || task == shardedTask2, "task: %v, expected tasks = %v", task, tasks) } - - // // no matter how many times we call NextTask therefater, - // // it should return one of the tasks with equal probability, - // // because they will get picked with their own probability, but once they are picked, - // // and are placed in inProgressTasks, their probability will all be equal. - // taskCounter := map[*ImportFileTask]int{} - // for i := 0; i < 100000; i++ { - // task, err := picker.NextTask() - // assert.NoError(t, err) - // taskCounter[task]++ - // } - - // totalTasks := len(tasks) - // assert.Equal(t, totalTasks, len(taskCounter)) - // tc := 0 - // for _, v := range taskCounter { - // tc += v - // } - - // expectedCountForEachTask := tc / totalTasks - // for task, v := range taskCounter { - // fmt.Printf("task: %v, count: %v, expectedCountForEachTask: %v\n", task, v, expectedCountForEachTask) - // diff := math.Abs(float64(v - expectedCountForEachTask)) - // diffPct := diff / float64(expectedCountForEachTask) * 100 - // // pct difference from expected count should be less than 5% - // assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) - // } } func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasksChooser(t *testing.T) { @@ -867,11 +818,6 @@ AllColocatedTables - ensure all are getting picked with proper weights test that weights change when tasks are marked as done AllShardedTables - ensure all are getting picked with proper weights test that weights change when tasks are marked as done -dynamicTasks - create 100s of tasks randmoly colocated/shared, - get next tasks in a large loop, - mark them as done occasionally, - ensure all are getting picked at least once. - ensure tasks getting picked remain constant between marking them as done. and adhere to max. all of the above cases for multipleTasksPerTable (importDataFileCase) From 1565ea1bc281611bcb395ae2a23a71a4233afe7d Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 5 Feb 2025 16:25:45 +0530 Subject: [PATCH 72/87] updated tests to test chooser probabilities after picking tasks --- yb-voyager/cmd/importDataFileTaskPicker.go | 4 +- .../cmd/importDataFileTaskPicker_test.go | 204 +++++++++--------- .../data/gather-assessment-metadata.tar.gz | Bin 10762 -> 10211 bytes 3 files changed, 106 insertions(+), 102 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 226eeac54e..6b773fb5c9 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -279,7 +279,9 @@ func (c *ColocatedAwareRandomTaskPicker) initializeChooser() error { } } colocatedWeight := 1 - shardedWeight := lo.Ternary(colocatedCount == 0, 1, colocatedCount) + // if all sharded tables, then equal weight of 1. + // otherwise, weight of a sharded tables = weight of all colocated tables. + shardedWeight := lo.Ternary(colocatedCount == 0, 1, colocatedCount*colocatedWeight) choices := []weightedrand.Choice[sqlname.NameTuple, int]{} for _, tableName := range tableNames { diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index a5e5760c5b..5554891626 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -19,8 +19,11 @@ import ( "fmt" "math" "os" + "slices" "testing" + "github.com/mroth/weightedrand/v2" + "github.com/samber/lo" "github.com/stretchr/testify/assert" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" @@ -493,43 +496,93 @@ func TestColocatedAwareRandomTaskPickerAllShardedTasksChooser(t *testing.T) { shardedTask3.TableNameTup, }, } - tableNameTuples := []sqlname.NameTuple{ - shardedTask1.TableNameTup, - shardedTask2.TableNameTup, - shardedTask3.TableNameTup, - } // 3 tasks, 10 max tasks in progress picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // all sharded tables should have equal probability of being picked - // table-wise pick counter picker.initializeChooser() - pickCounter := map[sqlname.NameTuple]int{} + // all sharded tables should have equal probability of being picked + assertTableChooserPicksShardedAndColocatedAsExpected(t, picker.tableChooser, dummyYb.colocatedTables, dummyYb.shardedTables) + + // now pick one task. After picking that task, the chooser should have updated probabilities + task, err := picker.NextTask() + assert.NoError(t, err) + + updatedPendingColocatedTables := lo.Filter(dummyYb.colocatedTables, func(t sqlname.NameTuple, _ int) bool { + return t != task.TableNameTup + }) + updatedPendingShardedTables := lo.Filter(dummyYb.shardedTables, func(t sqlname.NameTuple, _ int) bool { + return t != task.TableNameTup + }) + + assertTableChooserPicksShardedAndColocatedAsExpected(t, picker.tableChooser, updatedPendingColocatedTables, updatedPendingShardedTables) + +} + +func assertTableChooserPicksShardedAndColocatedAsExpected(t *testing.T, chooser *weightedrand.Chooser[sqlname.NameTuple, int], + expectedColocatedTableNames []sqlname.NameTuple, expectedShardedTableNames []sqlname.NameTuple) { + tableNameTuples := append(expectedColocatedTableNames, expectedShardedTableNames...) + colocatedPickCounter := map[sqlname.NameTuple]int{} + shardedPickCounter := map[sqlname.NameTuple]int{} + for i := 0; i < 100000; i++ { - tableName := picker.tableChooser.Pick() + tableName := chooser.Pick() assert.Contains(t, tableNameTuples, tableName) - pickCounter[tableName]++ + if slices.Contains(expectedColocatedTableNames, tableName) { + colocatedPickCounter[tableName]++ + } else { + shardedPickCounter[tableName]++ + } + } + fmt.Printf("colocatedPickCounter: %v\n", colocatedPickCounter) + fmt.Printf("shardedPickCounter: %v\n", shardedPickCounter) + totalColocatedTables := len(expectedColocatedTableNames) + totalShardedTables := len(expectedShardedTableNames) + + assert.Equal(t, totalColocatedTables, len(colocatedPickCounter)) + assert.Equal(t, totalShardedTables, len(shardedPickCounter)) + + // assert that all colocated tables have been picked almost equal number of times + totalColocatedPicks := 0 + for _, v := range colocatedPickCounter { + totalColocatedPicks += v + } + if totalColocatedPicks > 0 { + expectedCountForEachColocatedTable := totalColocatedPicks / totalColocatedTables + for _, v := range colocatedPickCounter { + diff := math.Abs(float64(v - expectedCountForEachColocatedTable)) + diffPct := float64(diff) / float64(expectedCountForEachColocatedTable) * 100 + // pct difference from expected count should be less than 5% + assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + } } - totalTables := len(tableNameTuples) - totalTablePicks := len(pickCounter) - assert.Equal(t, totalTables, totalTablePicks) - // each table should have been picked almost equal number of times - totalPicks := 0 - for _, v := range pickCounter { - totalPicks += v + // assert that all sharded tables have been picked almost equal number of times + totalShardedPicks := 0 + for _, v := range shardedPickCounter { + totalShardedPicks += v } - expectedCountForEachTable := totalPicks / totalTables - for _, v := range pickCounter { - diff := math.Abs(float64(v - expectedCountForEachTable)) - diffPct := float64(diff) / float64(expectedCountForEachTable) * 100 - // pct difference from expected count should be less than 5% - assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + if totalShardedPicks > 0 { + expectedCountForEachShardedTable := totalShardedPicks / totalShardedTables + for _, v := range shardedPickCounter { + diff := math.Abs(float64(v - expectedCountForEachShardedTable)) + diffPct := float64(diff) / float64(expectedCountForEachShardedTable) * 100 + // pct difference from expected count should be less than 5% + assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + } } + // assert that sum of probability of picking any of the colocated tables should be same as probability of picking any of the sharded tables + if totalColocatedPicks > 0 && totalShardedPicks > 0 { + for _, v := range shardedPickCounter { + diff := math.Abs(float64(v - totalColocatedPicks)) + diffPct := float64(diff) / float64(totalColocatedPicks) * 100 + // pct difference from expected count should be less than 5% + assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + } + } } func TestColocatedAwareRandomTaskPickerAllColocatedTasks(t *testing.T) { @@ -607,11 +660,6 @@ func TestColocatedAwareRandomTaskPickerAllColocatedTasksChooser(t *testing.T) { colocatedTask3.TableNameTup, }, } - tableNameTuples := []sqlname.NameTuple{ - colocatedTask1.TableNameTup, - colocatedTask2.TableNameTup, - colocatedTask3.TableNameTup, - } // 3 tasks, 10 max tasks in progress picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) @@ -620,28 +668,20 @@ func TestColocatedAwareRandomTaskPickerAllColocatedTasksChooser(t *testing.T) { // all colocated tables should have equal probability of being picked picker.initializeChooser() - pickCounter := map[sqlname.NameTuple]int{} - for i := 0; i < 100000; i++ { - tableName := picker.tableChooser.Pick() - assert.Contains(t, tableNameTuples, tableName) - pickCounter[tableName]++ - } - totalTables := len(tableNameTuples) - totalTablePicks := len(pickCounter) - assert.Equal(t, totalTables, totalTablePicks) + assertTableChooserPicksShardedAndColocatedAsExpected(t, picker.tableChooser, dummyYb.colocatedTables, dummyYb.shardedTables) - // each table should have been picked almost equal number of times - totalPicks := 0 - for _, v := range pickCounter { - totalPicks += v - } - expectedCountForEachTable := totalPicks / totalTables - for _, v := range pickCounter { - diff := math.Abs(float64(v - expectedCountForEachTable)) - diffPct := float64(diff) / float64(expectedCountForEachTable) * 100 - // pct difference from expected count should be less than 5% - assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) - } + // now pick one task. After picking that task, the chooser should have updated probabilities + task, err := picker.NextTask() + assert.NoError(t, err) + + updatedPendingColocatedTables := lo.Filter(dummyYb.colocatedTables, func(t sqlname.NameTuple, _ int) bool { + return t != task.TableNameTup + }) + updatedPendingShardedTables := lo.Filter(dummyYb.shardedTables, func(t sqlname.NameTuple, _ int) bool { + return t != task.TableNameTup + }) + + assertTableChooserPicksShardedAndColocatedAsExpected(t, picker.tableChooser, updatedPendingColocatedTables, updatedPendingShardedTables) } func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasks(t *testing.T) { @@ -739,7 +779,6 @@ func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasksChooser(t *testin shardedTask2.TableNameTup, }, } - tableNameTuples := append(dummyYb.colocatedTables, dummyYb.shardedTables...) // 5 tasks, 10 max tasks in progress picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) @@ -750,61 +789,24 @@ func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasksChooser(t *testin // all sharded tables should have same probability of being picked // sum of probability of picking any of the colocated tables should be same as probability of picking any of the sharded tables picker.initializeChooser() - colocatedPickCounter := map[sqlname.NameTuple]int{} - shardedPickCounter := map[sqlname.NameTuple]int{} - - for i := 0; i < 100000; i++ { - tableName := picker.tableChooser.Pick() - assert.Contains(t, tableNameTuples, tableName) - tableType, ok := picker.tableTypes.Get(tableName) - assert.True(t, ok) - - if tableType == COLOCATED { - colocatedPickCounter[tableName]++ - } else { - shardedPickCounter[tableName]++ - } - } - fmt.Printf("colocatedPickCounter: %v\n", colocatedPickCounter) - fmt.Printf("shardedPickCounter: %v\n", shardedPickCounter) - totalColocatedTables := len(dummyYb.colocatedTables) - totalShardedTables := len(dummyYb.shardedTables) + assertTableChooserPicksShardedAndColocatedAsExpected(t, picker.tableChooser, dummyYb.colocatedTables, dummyYb.shardedTables) - assert.Equal(t, totalColocatedTables, len(colocatedPickCounter)) - assert.Equal(t, totalShardedTables, len(shardedPickCounter)) + updatedPendingColocatedTables := dummyYb.colocatedTables + updatedPendingShardedTables := dummyYb.shardedTables - // assert that all colocated tables have been picked almost equal number of times - totalColocatedPicks := 0 - for _, v := range colocatedPickCounter { - totalColocatedPicks += v - } - expectedCountForEachColocatedTable := totalColocatedPicks / totalColocatedTables - for _, v := range colocatedPickCounter { - diff := math.Abs(float64(v - expectedCountForEachColocatedTable)) - diffPct := float64(diff) / float64(expectedCountForEachColocatedTable) * 100 - // pct difference from expected count should be less than 5% - assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) - } + // now pick tasks one by one. After picking each task, the chooser should have updated probabilities + for i := 0; i < 4; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) - // assert that all sharded tables have been picked almost equal number of times - totalShardedPicks := 0 - for _, v := range shardedPickCounter { - totalShardedPicks += v - } - expectedCountForEachShardedTable := totalShardedPicks / totalShardedTables - for _, v := range shardedPickCounter { - diff := math.Abs(float64(v - expectedCountForEachShardedTable)) - diffPct := float64(diff) / float64(expectedCountForEachShardedTable) * 100 - // pct difference from expected count should be less than 5% - assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) - } + updatedPendingColocatedTables = lo.Filter(updatedPendingColocatedTables, func(t sqlname.NameTuple, _ int) bool { + return t != task.TableNameTup + }) + updatedPendingShardedTables = lo.Filter(updatedPendingShardedTables, func(t sqlname.NameTuple, _ int) bool { + return t != task.TableNameTup + }) - // assert that sum of probability of picking any of the colocated tables should be same as probability of picking any of the sharded tables - for _, v := range shardedPickCounter { - diff := math.Abs(float64(v - totalColocatedPicks)) - diffPct := float64(diff) / float64(totalColocatedPicks) * 100 - // pct difference from expected count should be less than 5% - assert.Truef(t, diffPct < 5, "diff: %v, diffPct: %v", diff, diffPct) + assertTableChooserPicksShardedAndColocatedAsExpected(t, picker.tableChooser, updatedPendingColocatedTables, updatedPendingShardedTables) } } diff --git a/yb-voyager/src/srcdb/data/gather-assessment-metadata.tar.gz b/yb-voyager/src/srcdb/data/gather-assessment-metadata.tar.gz index b6d53c6a93796bce381f8760daf047fc68cb4419..c6d9f40c96d01211937b12744e9309d4d0afa18c 100644 GIT binary patch literal 10211 zcmV<9Cmh%xiwFP%KBH#<1MPilciPCZ=>B^76}<@=AY&fJkE}(8xgj7Y+`-@hoJ>w8 z!;wM8Isu7D!j5;6^V@G#b+^=d0ZtsxWX4)6G3sY^cXhqGsx{NS3&Sbtu4}mN!mzy3 z!tnH|?&+0(-PTjBRv#Z6F#3D=nEqC4`~0tX*zW$};a=_F@bIw8s=JS?`+NUl2e$^n znrG#Dx&xr-3;mrw*Hg0u$-^l&3(Xp9PaO@uK(`g!Q(LfuN@v7?*EHbzy0w4@#{a- zMoeqH=(GOOQ3hX*G_C!t+4{v#c%e1ho!)cS@6qdVbMVXQ32O~sLJDhZ-Oh1mqz%{( z`}MRx()?;C&F9*%^IsaP((7o@IUb($`(4=%)H&{-j{Nd3wL$0THN9p^C8m8aJ!V^M z_-mILj$=D6vmMqSikHl*7g|q)I#@s(4Eh76{ni=bTN(bf`fV*J(jO`%j+&kB=|H1S zhQ{2Ocno#`L%e3D<=OOxqdPa}23xtNHG}lIX&LO&G_KqXA4Te}n7EfPp03s!F?Vud zEcCIZFN_^tLHy>`#0%CPW8KWrpnuE?83qq;fqvrSmF+9ba9DE~YE;Qrn)g%HZv4cc zJAA_~gRC>i(a*+WRLrBC|JU#IqDAJ0ivvOokf1d%L4w(9hd>gUK?1ec)s9B-)|a~D z$u5>&xQC^e-oes~wh!wM>l~fJ2Fw4joA;ArE%o{%*6sYFv3z@wd$?7>=C9F$t$wqs z4O?0Ppuz!}o@rZvNi^|0dF}XQ^g3UpCOSP<$PZr+N7`|Ihvf&_5kTE*Y58KN_(UKZ zZ3@KR%&qVL&Gg^2rpAYoXO}q2FZAUSMzTHi-$?&!`v=u1{jXK`?&<$sJoohf%jkc~ zu285yl^An!y(8XKRv>c}s+GsnHl9~YDonWnmn2~nu1f{DuCGL%(5NC+@)hKbmyAs)|9O*^Zm+T?~LmJv<*|i z`5SPj@x$b`OJteAw{UYoE#{+!#rnifnC=+3&kV=X=ghr!J!9dQnmC5;8B>CGfy0A; z@OpMFE4N{np6RZE?L%LF#ZaTLheh#`@xj|+$bgxBE)w%@P;#ML`piJChxs0qmMwmJ z#`_m}HuHb`?0Q2-`TxOg)cyy5tB>#b|6M%y{Qt}Nf6}fV!Kj05G^rK#=@ZM3^Z3UuyQ{M= z>0o&JY}-EEWuQ#msDG_H)Z(r7UUqARXuX}Z1|dTc95mp8ur_MGx^~JMOX(maG4TR` z(rXl9)WiVn zSuu<}B=2zC-iagXP9kcBC7s(*(6P1C9<&$v5T;XzMZy{DzW=;c(K}Rf$&m*h&2;1sBVOd#)Nyp!lEaW zUpi^1uN9!AwOWi}z1$sEE#7p@)P|SR25E^@Af7MA0%>EylZ?-Z)f3T7SYeV9n4-dY z%gC8&T!o7soT@96(m|4%7->0CTZ~UwBLVFafO6LWl`NTj=|OHF%p}M$TOHo=LYCp8 zWx0W;1bIdTW-?z*4JIgKX{y%(I+|>T-I|2nP77C{nLGd08tV|HBb5h8H{O8thuUlx`;*&Q;1=oUG&% zk5axAtDr+4tITAq@?Bx3A8?t&Lup{Lvf7~`@XR0~VY#EN#htDE7QPpEtH^lea~qyk zYyGGE(%-~43Jb-yO=$IYLJfRtoBOuwkCg z-0Oc|qyMS90<}MEv=bCYSHzjYJ3zNZVP)bjiCnbUp<~dnv~gv67eJ#N2Y~}303h5@ zuSAG|;f&9&J;Pn2Sh`>2W&{+#m?5o5^fL-_ll_2FYR5c{!tJP%WSU8OV;2d&M(Mq* zlVwwl;%nT3B0#0d9d4Nxjg};|ijw(-gbL9lNT_RM%rtq{T_?{*CML<1C^`+d;Y#Et zOTRH_Hp#z{+5UQuA@?o>>J7s09f-Zd)8m3b!D0*t(Qc07g25)b#m)@2Sj|0iIX8Sq zY$9DQ0zgolt%!(E&2w`4@D#$wrYi&KBf%4_1#@i_x92)Ji$G@!PL*wIy_-{-)E#UP zE_C}CpDt`C5U{^(OvB$_$d;?MQLO&6C9yx}*)0EGpOr4{YkdZ#cuFa~PT9Rk=wp5S zf9>$`ekA^XZ+E|XFaO`k^Wfi=Gt;Vobb66_0NbWzFRvYQcHyzYq{v>cW;#;-_Ol(3 zYbWK*11i)tCx+!37*aF^0)XOebEyMQk~urdDzBld&L&| zHM49Fq-4VMnE7VK$nPcD~vW-w*IVLhMkizYA?@D(JYG-N!H8g=ChNexz2xC_Zz{CNA%>w`J z46_V~eP_5stqct#RPWs8?YK*0VxF550LYrHV6(EBeQ7ur#ef;k!gO6?x}a=OJ9Bek zdOBysF=)z}j0;*z$;f1etwHBxG;VhWja-3ErYHeq;n&o3i1O0wM*#4}c-TK3w6uCD zSN)u2#Vq^bXI7aSmlbO@pR*r+V9Tp1PMS)JGp8xlX;6Y{g{^SVeQOv>i_eQHO%q zfV!ZZbPm1RSH=`(wkb>3flZ|i2r*Rv800aL2PlT!@-{JngXk745R;HHiv$G6T`dtF zjHz=GUdz5J)g2iGn zzX8bF9Rb;CBW&h$3y}6ykUYyB>p;8QO|>-3T{(jM$pr>w$gyGD&ac_~mF;oLqF>r3 z;N^t@+le9=myW@=FjR0?h6~8WRLvmP6=E)B2*edQjx^-z!kk=$_JBju`I$k|I3OS) zFJmb?%WofkLt)#w22md<_7!Ohv`hSoOjZP<4r4jPy=Obv{`#33(G*P_6PN?l)IHBw zzze|1BS*O<$A`c?XQs!dHfRUPWsDCnHW-BVK?k;C;5O(Fnys!jK4}hzulj>_2T^FYU7)@ZOS&&#s6Pq3&(t7uFvY$)MV^ej(Bqh2+0ZU!Ad zQ#vfA&t3y!iWmsn;L@BLQ`zH#EGx}DwwKVCWEj<3U=s?oyBTAybUwS|&T*>JT5RNY zGfI<@ua65R{)+*7g_P#YS8 z0#CEb(BI?=Qv|%UVn5~|zAimnlpapm!x#03$MuK9eDN)NlcSd3vYcol%RVs=M6{9) zvn-YSl-}CJ4TeRw`$=OH0!){QwgEH4pNX-2l(E9-wx~@!Ad-Um4_Rjl+iL+OT>pswt z*7Dd~p0@{>A6_C@ou%$TC#k|9F2KyN3Lzab~9vg=0nQG7ukFztf}!1E&-zsta%K}vC;tBA$_D7$#@=;>$L z?+y{wd^Rv;1?$krPS4CbU%h(HZ!SO|1(Z|NRVDW5SMTH=!dWyFFgXVg&1{v|yru@`Rb}rv~dC!H*w*EGCo{OHs&~?0;E>B%Mc% zN?J*LT+o<3C=WYihh-aR1V}52NZ8Qd)ZV5SnYpX8LgkM)?04_&kDvITTm>4=?ojoV z)<5$N8$cdXL2hpZ>JIvfk%Tm&ZAS!RJnNlZ9rS=tZ}-7s3gJB{+8D{wxHFMJSjVEM#v*&nQaQa9!_ zt@ZF!Dxz7KUE&# z^YoKA6q1!gjfx>6=8*pf4A5IS;%+D|L~vGVTI|0$LzF4CpMTg*l*<2`U4d{hGdwzJ zq#Lu0DRqK|XgQe^6jfa6F`bUt+b(>n-0iKQrdM^)>!JYKd*w-(Bs4Ug+f9ZqU9sH9 z&^SNC&NEmww%I~Dj8GE+hh<0SIt*+I*jtDo7vTk!$riH;dEi3}7ML)0;$HG20krS{ z{jlf!)&N*#33(VGpkBXQs#!TSzs29JWChJ{WTJbJ9;&pVhR|8Xo(3$%d&~T_f4~L? zKz%7pU}x}q zn5-N+ulE)97`1XYHQGqG6!7U=uH1{YLs1V9KtIE&=2TN!StYH>rP|Ryv6*8mS;;WB0Tr;%u6HM0MRx)#DT43*`6@3U`;GpJ9*Aa zwj0sW@WG~Zf1cnOVT*%GgzQtla z@VSC&Zh)0cE16ZcPlV73xSI*qY+)$pfa`SVf=U%A_<@lwFO2R&--m2r&<2v03l7^V zkEE0B6E?MB=v~`h!pI$aHM?Mj4jXUyYxe~5WqsVi#s_191aN$epF7V^3g`2^0^AULXesf(p5&(z!_9Wq5dUEsqmtzVs=bUn2tec?^O?30$2U z(S6W$fx0aE+?^VO9TI9cpA9ug5~IoTyB{lmKOIc1mQr}J<6BQ@+ zYCfeQ(qZ(3Sg50rqhq6RzDU#NZPi5~W9JJ&9 zRpHJ=I@AVo*(496{9)^b zcHER%%7&-hc4vsLq2pHn_*m~^*8m>f@s zGIw7z+Z;8=c)!{BxOsBY={=YEFPnqbi{_wK2bMRv(4C!LECuRnb~X9$gUv8W5J0`Y;tb$ZRg>v0N9BA0g`&mMJ-Q7bw= zQReoy<1Q{z{lj7|t4HBE=?-IQepBpKXY_*MEBmj0PgynL?gX#wKR|t=HM;#)v%5t- zj!ZBO{Z|r!d||3hhdKPL7}XIG`G6IULig24o-h)?INb8|*;%Cm+6 zO4!LGljr5?S$W21IUw-3pFVzMWiFS#uwha^fBuw<5v5PFEc@Ljl+k*?EYxZ~?^s{! z6bO`BEg)WL^@f~|VjY#Br4?_RE=tTT;4vt~Vzop{x1qj&abKu~2sKP=0|p7BJ)iJ- zjgrdLo&iBd*GuD=-@3~?PNtXfCMq940&guh(RB)HR;I~)$ox+^zIj^x-cjdyVsnNy zjkc4EZx7IAb5YcnI}r!t3TGRxqZ!j@_`Q9CJHkj35YjO z-+s=bDiVmmnC|P}{DM+E3d_H#ua-;0spFB;n>u>E;qO2*;G~2M1nFu4XTqQzP&c35 zY_vM=VW>~ps5K50eUDfwbUdN9`rZ38)G_zObWI3`BBHkH(V) zXdHqzLDABCR`Mh;cBo-iI#XL>rAZuNSgB;$;|m~YhOWT7N@M}Nz?OiidngTq3Xn3W>1weY>+Tqod2~B1fo>0fVaIGh zj}RPz%DDA}@QT4?1Ows?CC-Oh$Y(KVjFueKcFdWHUl&lEUCz-AgdTmu<(9lDWJt@> zUQRuw*P)_18d8|29uR^rSmb;X}Ww8jfN!c;xsi9{Hm;OOcq`Wh(^K1GxW#6>6rT5{0~&aP-5j~V7#U8`x8 zYrG$OtK}F?(_2i%(uEnF;s=RzO9cj-!6w`Ql$%0g5gaDa%OBN*6_CqN!qk}G0MO9b zlzSa9Y{L-H*7FNPN4Hu(f|MdAqlv*SkjNp7q1gQ%Nd4FLio#oYP!iW0f%gM(;6xGF z{JB9#U*g(;%gc7o^izP&kogSlsjAy$z!V^v< z@t&GLHIiUUdPYax>s$Ipfp55Qv93>j*{4TqAxwDH6~<2|2Y1%WOG#8&qVCtg1-!R==4v9!5ZL3 z;yo2$C%FbdHFb++R%jdN`fBd6-5L@kCF=0x`R0Jb*gkto4uiPS>IeIKwaVgJ4lEzW zw@rbAAeuTWtaMoYM+Xt^uaD#VD*k}ggB|iSdy|NFZ`Z~d9{k1N!wFC7R0%`GUN?jx zKHwg;v}7rikjRX?gGB_@!S25>elZAKOsjR9VTY4dP@Ql zurZma3QCxziMfyHydkLHOF$RW+|U`>4=NW` z<8on6)N1~I5g;jdc@S8IWk}yCs&G=(&e5=u|CATr=T5YJ_R~*4k+-i8jvBJU2R9}! z!p6BST`svVvru$VlUG}e`w#F0}nHhem-^kECg#jSKTq*aSt2HaCT3 z!%LOzpliCX%0)#^I#R6gvpO_K3Ne!`H*~`lKQXI_^Q|&#lX43AXz;i~V;U#s(AvuQ zzpU`O(ksR=K<1By|A-yTL_8}L@fgX=uJ?)pBocf@+?GH&FkoSi1x3WfcY6V}+;kN% z!~!Puh--LlxUh(HE5f=`$u9iKt|xdSH-6<_jjrH`4G3S0(q2g>g}#TwJ2PNNTx!DC z7t3^kSJSjHc-C<};XyMg0 zfxLciOalCPIO?A?U;!a5>VwfREuc&KJR5^4OuVEYpeW9bxB+h99#qBJn8AaNJOVSK<%NrLZ47AtJu&^|Fj%Du9c z3%n_|Kz8ZwgzFM>9s{2n3$Bcl#d)=o8Uuw}nUY5ppO!xn)y~ncb6Z5C!ha1kBTxv* z)ap;dSDa+0XQlTm!!ZrFhrQSBs@XnU;X>WLXg1@x(_}5cy@ETp^hoix1 zYc%9a9qCvpI-S3yiF9oE&g35cVJcP!bxO+JAn&cL;Eq!S|5{u;zM61Rbjkx2rseqj z;Uh((Sf&(U(iYk`w}x%L>RY@=IeVc}Uph@f2v41+O_GQrq~F5aeD${E3^XE^ByG)Y zHz4of)^=QEyFB`{nBbp3e8bfz5d(`n)Ssu<}EHx;=Hc^A3` z&G<)2tL4nmr*yn3i$W(bRTt*$!f@7y+1_SUt-2HeqKZ+ZAZm%XRkE%5wf`2u(vnYD z8r-mskt(GL4=!{08~<1Y{RHqY&ABl%L>AZCQTxJ#xvfu2m!@9wU8hTqZDTJK^&@;k zMGy!E|K#IwAHg;dqf-zegx(SKj~4L_c;9b?$NFbG|IfH+aFYm-4e=k<{ThA$p;oQ# zJ+2+@)AzsczyEeO&kf=~NV+noJHbof|6W&^csVY3C?4R`Gja?s2y+cuTBNDQ_n+f- z8F&?@7h#C@3lxgzuN`x{hV7Y8z|_B{3+UktO2=kGs7A0obtrcOM_@N8-N@ zffn5F|GRkZ1kHeo-}SRFop{Xul1ikv&}HUEm(9+n#1gTCeD z7C{1`vLMQ`y9L*zKm&A(0ojh+OhA0Ugc-7@AeeIbmtE+WbniBIge38d+%@c_8$>95 zO_);}XXC3|!qqKF8hj(xmiXP^?dU(ZC3!+pFWry@Y@+|WyVWTDzyJQ%9X&Uo{}?<8 zXsvPq{y{yC#XGW)$_Rol?(uBzY#wccf5ua70}82aDBt;1sT!w0W{9eBN*DgZx}-_; zfpws1@$%50UtX3*7^0;E`Y5J?Y6Sz9v{O!0dVMhsSYH{etS=`i96A}D{@bH0DZBe# zZvF>6H@W}8v9C(p$K!VIe>iyjIQsqfy~D@%^#3j%kc?W~(Opl9bf&-}e*AA=F~0bj zwLGJs9_JIG!q5#t*n{s!2Z43f%4=-+6%^mew znD8OB>a$!f)7Dzurl6yt8UV$cx2)b^Q3@(e4#KIbkdOA3FLJW_Byqe?l<_*o1n>SF zXs!OB{Uq~DdoEZ4ek|VOA4kCmOMb6w;6oX4>!d12eVF^!i%@?cCleP4jjY@6pZF?6 zVn?2ILQ!4)6dSYXE<6b6D|9L#UD1;H^-id)c&zozKl1qB*6y8lyPczgTee_;mj+(r zu)|8bunJZBP_?{iXgc;0D%76uB^SM0A^c%uI%O*g+oU#lf);q zGIDh+7ae*1F$BAfYZ&wi>|m3Hs%yV#ty9>J+|pl&O=7;&8)^gm1T^dqb)%{(yt-(? zjw~{|Ms3In54$^z{<+0`e4SJjDX(ABgrdoq5-PZ9ah0!ipe~G4{3H{8!(X`^CoLNH z#grE~@at$(?t0LJ`*7FX^PsTC?@MgD3ICti^VPy~`9N>a{#UE*Melz)tnMG)^Z&be z?(KhnOZ(qVE?ruxH!p?Asyw7=^-p`F!lR}OW>jdhS@ zx>LtqE{$n%Eo8#l_vZj5=9AMq!EU~hLG>n=Cw&v!=r^@lCh?8$i$DI|pD*G6m`ir( zE$sjs`TzbQ^8dYEbO5N9gTz(CIxd@B%^9 zNUcc4-n`9|ayFo^SfFbccZeR7mm3eMpj|)*KsvAw^uB)cb}e9dk$0)B9d&wIdp&## zRCs)4aYudooYCFjGIW~*u%fSi$nr4&Z1fZ2P5tyU%lrMR8R0$cXC`=J-ROyEgn0mmav70{?Ib@CU(!KXWM2g1>an5BxQ; z7=9;UehokV2dIFoMnJ<4gc$2nUjVsiD=deptIKE5(kTd#QgAlq;sm=!kuZWpU%XIU z_5@A{^hbdskOd+;sqqCI`=<9D@=pwfG-s9TtGhl>j1zdqmI8p%-n|&`&7Paz|LlTJ za;hQwoZtQ@@1{d` zS>UXb1TVKJF3jO_0wxvwNM1$YTfnw9WvPWZ<9_T#?5oc|{=;u@u{`qV7dN`Bx0i3e d7@6~udyByRbN}2w_s`uw{~rtoP+9=m0005B{E`3w literal 10762 zcmV+lD)rSLiwFRq!J%gW1MPijmm0^Gpt{eQGpFg#)1Tv^k^#9`YqODY3qh8|n1V7; zlJ#Vn&Vnc^Tabw|lUkmn{u?u2=ge=I|2Ox>k{P){l~h8uL>$XSW^6Yi?j8|03&ne% zJ(WtOgS|b%{_gX?rOGb=5##Xw{zX!uIJX>kVpHes^mC!Y4cC9K1j}A7@5B1%WdvC|b5Yn$pk725wLPyZe>$X8qTZy!ChW(<#l5tm$HAI{C5g>UsBiPMrcUhWT{i zd_o^EAN#wziR)k9-#ZAde`N;{<{l~C*(jH!Z+!jx+JI<{XI;`gJWS!kp{6yT)EmEe zFP>@jW~=j*bUW!l0Kpf6~u2|O*~-TA=XVD z_PR$Tn<9{S3+yK*w_shG)F$YOu(|^I*YioytXzO#`c4ZfHH`v46P3~ashTDhrhjk85 zVU1+ID`&jdk(N5$0cp2>(MYD*+j?-Lg7sg+1smOZTkAKpEI@?=GF-zl0h8FoZ)dcl zE{3bKAfurIHv03eZKvZ)YTWk3d4HtP%6wGqR1jbyefK}MeFv%*IE z1bAT&d114ARBv_g*WpR0F=%x=`6uxO z9#cm!yb}IefH*ZB*VX|^{K=4sO$yG1U&sj*V6#GE=faaNsyH*k(o8L>K*A=EO&pto z6Obx^*v7=PY{cBTZOnB0n&{@3EKK9=f|57%+6QgLRB-tQ(rNf0e(hpKCde%$E~rI3 z)G$ex$r+;C6zPoGramRkwd2y6S88Na-KAp&?F@$p|KRcTT2^iXm7eMj#rB~uuVSE4 zpkYybK(E|wf&!RWXX0hX@k`EhQ=iZoHCet3rDcnsU-14(k|+PI)Bgls$duSS75?{D z%KyD`*#56n%9YYa{;wl#7ak|rI#4OXo&*|-iRF-Ka!l~X;d&D z0fi1=3Jy{!!j(BzA~-HyNbTtM7P=L0p@)H4WujZjx?&J{n7+etdpnA#+p(w>)O7B~ zK*!cjI?y_)x7)E~yp)K88hb1*DH<~3Isntue#fCVP+?#i+v0oUryU?e?3y39NYW-jl72a4gAvi&(tWy^tMOoUT zXVj+Vh%zm|&kcZMNrIs54;W{Oo7wp)H)w#tB+g{K?2KasIC!7lxp4AdC@kmZ%?U;# z+Tu@f4Fv^E@(Kr;am+f;mpDFSfn@y7AN95e^LiYTo3z#x+W)hy%lyb%nC@Nd`^mJ* z|GQt_57~d^-TjUIx0dwn+JEXko`Uwi5yx0%g2>?xP?RLTE0pp5I9*!R9~k2gh>M_# zelu$)QDgVua3i&s0^ueA5oyL3Q0i~ZByE^A%#Aj0c_EWJ=vl7eEMy!>7G2(2x)eLE#XF zZmF;7R*2zibx{$&6lc^;fOTSzsyK!8rj=^be1k>aal=?0{&V2fZB3R{co*CoS}W?s zs?^JNUY2Ue-#7wB@M1gJf?bN2^4Q6WeU-uuc2+X6TPYulRnVb_RYp8kc}HQ!SGdf< zq15oTvf6uBy!ban{9&zwT(-|JqI3T+Xx&m0RZ8ab|pdt0%v%7?NVonVd;F-F(Y6N zbb_ZQW6v4+x$)dD^M^O!9nz!123~F8R;fDrDV34y2gA;y@S|{bvXzCgW`0-nE2Q@V~0Lm zh4GQ`f+Bxp@c3&%91GR$sm>0IKxZ?~l`V7mF{dQ0+b0mNbbA<|tZc_Hu&?$`!`H85 z%hg(`R)1+t?8_ui|Gz%XUs%`rgxY!jCO^@U*7LgKP{%=f$j_*&qoP!B&TrGl^85dl z{e#```TxCxo$^NiUq`zCx8kW`7C}2bPu&NyX;|}X+nAiYBsG8}!;y zA$6Y>Y8xYJIux%Ije!B6dRw3CAd_UyHhE5M2iQER6iOtE1=1ob4U~s%ytEc%re71& zazV=mQE+ZJT<4}&BRa>MIKVKXT)&4xpb_g7uL=&(5>ZOYmi8^8?>5Qu?klQ6>A)dDbaAYjuVe_Q=D z0cPLoZ?jhVlm_aZS-c%*PDjR>F#>?h$pWaDOsorPo9qr4wP%LoFrf>^25V<(%nVoO zf;a+AA(e8#YRS`7s^93fjt9eLt5@5~lF=A7fHeFX8#d#-?C~7{_-xqkp7a`8HNRE* zkS4h_`R->@9Mg-UxtLDLci)lu&4nuIJ{yg+cA#T?LA7`5yUjo$4Y9w24HhWsXk3Wxl-9fhjBD6*ZjpJPQ zS+_rc-{+R&V%1#sxZA@Q2tY2EZFD;wP&tOp`k;>0!6<(Y)W(+K z6+l+j1Z1m;uo>eUfV886CE!WQgLE-)BFwgqH6y(VuLmdiPdeqkAa zm**77iQQnF+mw?qRB#s50pud8Wf18KF_&cs#1%M>Jmm7+7@Y@bz-HR{DP`I?ARt3t zLJ}y;Yaf0?VavV-RUbI^C9@V7m3S42tO!gU#&U|ZXW7{P@|ha2DH_=Z2nSYEcU?Mz z2Y{2`9Oa%I4+7(i7%mxGU>%^8p;s_A7=(7EgIJ+R2Hjr0(bk5?^?v_Fx7V!wGzTy* zfx3SVa-IyosCpNOR^F?W6A`Wcq=AS=GS7POzamay(Y1>H+!f{AWm2i*DMiLb1p-+h zqvlh6#Akv1I=804r-W6ekMLMOxzMM8E5H~Gfmheo!iGjJ4BIlXi$W@$0vXXbr=vHX ze97kl@8-msBfh0zVqXE@FQ-YZM$$YlON8WsKp=w)V@$`g$NOoLpSolxrY|WlN_QY86llvSI#oKKthsZXs-zZcTji9} zWNoXGN@%mTwZmudZzMm4kt7t0TSDeJt&*lkRGFmOnJ}dhfuOOpiPF)qgT7FHoL=$Q9p17c*<#m{4 zIfgV*G@;aoRBCRcQ4BLMTm~w+4XhpjqUZ!(v8K{$-Up<+!-dJ!iG{-%0247>AfRv= zUg);LU2>dzk7i>|jbnPo?&IrHQgKyj3X^2>))=Fqz+zB!pK)aqSqK#@mxw*3A!eKU zIK}!F_#B!Ec*&QHy;JxFgVd9h`V`P>>wF}np|uMryY5S{}zQ+ zKoLXDhp9#>AFo=S!!BSIb|74ILXC}qj_rR4-~g(0+)_qs0V+9U(CHJVKnZPDT|5ar zF4e=w4{`m#zzF304RX?|xUQuiMi=xx(XLzoo^R23UHa!4q!tIZidg)2vWxo%)H9xUV}Cf(5~J^bS)&)tbq#3BVv%BYNT@rKfeDy7gJU&MRv;||3!*S(|K4c zCY8kGfW_=Wd7zMOlCGf>AgL%KVNHKkd7WHj;w(i5N0e>{Hu@bUZN zLrjm~i%lU}InbyWB4Q3Z{{aJZ7q*BE#gzyyDh-qT2N#I)P37nB%CTDce~?R1E+*7v zJB=)279F!r&=D;sbA+mj!}>UG9c%qTeQ?r$ReCLbC})NZI+N6P?S(lMzaz8A-iGH6 z9o|FN8Zz30%y*S)9<+-4oEQ=H*?5KC#9uO<+nNZK5)6F12!E`FD`Aq*u<5+nWO&*Y z*?Jck=LeuXN=gx83)nEiOavU39i8bgurXk7HiTRZFIbs$F0HT!J~V%U8ODyB3%(_Q z9v_se``wH;(7Z-Qx(n|El~&XcHmjg%z*3A`=B@pG z(xU+Ng>ZqL!taGkxoZp-E5Wv9RTEA5>}aV8&pqv&mIMraFp5U!%!ioZy~9a`g`%@l zu@f*_*lfSvGuR{C%30NDE!k7Rr*FP+&X*2F-9P~Q45ylNP3dKoyk;xWkN%!aY&s`- z>zHKzc=bo|@$2sepHasmi~!u35iJAe%Adjj{W{hm!O1p}6%fRQbc>t#?UPnYk31c~2`qsh1$1JP&~HsPNo{+#T+D zES3Yml~vshu#!n7(+c^77@dH-9^=gxj&csT&Nf}pr~(7uchY5q(_Ps6kS!eAz|yw- z&9>r0dHVJd8Cx*)wq?y>ZGPs<@y-(&ivfAMli_?6c?DDuLt^y3Okz0_C+!d!OfFvxt%!iv`=~XHKc%_MIeaP zz}2}C-TPfXAYCOQ)yw+t)mDlm!Nh5XhL zU44va&)6?t=N>+my+K_fKNqi-uup|{ zV>a56e%1MY);-e1gr_~3gDj*AA9gDK zgD7*->%M3;s}{cWejT>E^+8p5;pd=B8h8$6yQwv*V|bY9Q~cdJs<-{;?g=}c+k^Lg zn9*qMR%cMf9qv_*T4VnV2YP#KrM%!4_fmVU}z~EY)dhg{h0Va{lyN_lM zT1RLV9UUuk``b|)7pZzRo6728c#hls$UCno@}f0(M(~mRz1vY%O&oWENAeHgKH(be zZlm75K|PL4Fb&-o5`jErs>wEU_+BxZBjV-*?x^F24!65>v2y{&}N&WEQ{Z@o2y`QAX?;fKJ*8^dp zR`X=X@>(arpwwzU^Gd4M=X@0Fs0=Nsc++rDV{U+qK_wQe#d5k8^}WD-ffhpCFsTg~ zB#icK#OF24DieDK1Q}e<=@E~*%R7$em(eDQ@7{rIE!6So6x1w^B|R@Jk*W$5Op+Lu;P&u-uU-*nipSc3vI{{e&I)zysR` zDl_jG5U(D;{*Xpf#8-jwy=Q;(0HwGUmU&fO%;(gu;+E5^DxP}7-@awQSqXCxB%1|X z2!nONy7}Ow!_|?7p+3t-t07)DKa@k^0OmQ=pqNkjH7Oq!c;`QCtG)O+K3BoyDLYC< zUnS@KXgHdI#UXeT6wSXSc~=5s1sW##Q?(_MA4L&{311BsIlw*vSA%18!6LUVMHavdVhM!0i`p=l011N{FJ|+h?hL`0$782? zujKJQmosz&;fX$R zkv_uPwZBPTfbZ`83;e6VzqjxYpa1X8 znye27J@FH3{BQX8uc0FUe@g-v8b|{qwz}|MmYK{h!(E@f{3u zlaf+^{*U#mXH2>j@01U$r2ld$ME^T``+N7u)m;uWDShMgzf&SJv=?jTgPkAt_ewuh zek|-(cK6}G(hsS<1JZ6iK@UUgxmLK+UDqzek!!8~YpXu^J1uwj58dBh{F>VR5#1`S z_RD3J6dK>ek^Vv{K>wdb`rjb_bLAjJ|KI&*osJaPklRIuqZ@#=X@&|g;J4Imp{Ahu;heAaPhS<$&H>fI%m;afAoTG>D~ zBf~@9vs=K1BR-6llWmFbq`jfnnI(N2vTsHs%SyK5pj=sNCTpS1WE=X__y<(OP$J5g z0WeH)W?{ZDtxJ=?j(#lQ2D8rYW7~KidzKN?Fn1Pi!J%)5?qWW~t?3&~#oU1z9peWH zbyEcfo53a=AC%idVlp@ku$R79Uo5~}3cMK8=`8>aj7^E_jMp})fVP^MQ5~<saBN zgRWp?sw^Ej7a8up#|>7Ds-*Rx*JiI9<-F?)1Rb*fm0LCQ$z4SNSbqOswf}cZI~BbD zzh9|rzW=_Sw6Xuc>6Bpq-zgthN&lsT5d9w70G6HsCf-j>CcRan82a@S{ zo2(cQKkWxdH0Dv1B|sKOXvbREBR|3y0~|du!;wCBq1KQg6W)*GsUe$VwY^Nl4(|Aq z;SO5e^5nKnm|H$}7e&C{{T6Y4A3ZxfOG=lQ0@%}tU=e74 z03~oy$u~Mf^q@*nH7*y-iCWF07XgwI%aehXod@iloC+ta+B)plGVe3u_PHHypZxUG zPwcj@2aX!Df)8%UZV}ebba~~H-(?ntE?V+xi_!Z7oJeISTnFSW$516(ShZW!U<0|Q zFqAjmo5;mY!4lLqutx@cv^y2@?`v5JS56{_%BF+EdVvO zLYBp2@S6-YBr$+bX+kIl;3LV=7nFwi9UogCz2RS$bB64`JxWwS)i`fX-mSkK;YA?+ zEg-@0*~c2tzUI%4_&9?&Wsy;P$UG8QEcC)q?`aI1m&U&<*y@XV4=)qF(3xAX8W;s~ zJ*G_yYh1XOf-Wd#XmeLsI=EEn7G6#FOu4Yi$&M5i`B@!wM+!BQd2U#QYkp#;D9*PO zNRt_-kdF?JOLV4jW)7__=)aTf%VH-N!2p>*6!%9gefR`JN0)|+?xE^s0uc-rz$f6=HD;0l*KmO_oevu2H3`AGphz$r| z3G-gbCWXC+!#kxgB(62#V+6g$c_Or+u3RA}B|YTpc(*A3HiN+x(WPI23c_{|&|Aa?S;bKWhAmWLRkq+v z#HbKN71L2@0!0}u$<+w;SoIoWWp5jECEhU*j<*t5N=#uu_X*ciHIdvWT#r)2vZ#%U z-m^|_bzBxi$ABiYI>ZWAxm|n}hVzNRRg$>!tpcO?3=NshmNE($bAZ6p;iGIyq*D(P z#SE;aImspx>zV8hLt6wEUR@K&>vo1rfgkn<-QyZ8AiN8o!K@i3@Fjhkj=&UTUh*EG zD9)X@K5iflDr1PY7Q76&yzK0P#2x>Oyvz$=__qU>!QV$mjQG~W3Wp1mZ#<6=5~&~x z<9+T*5^_h1SmFDI_L(4LxtEsm0WTI?Kz8YD2kSE7JOnv66jB){i_2=|Z3GnVWr`nF zbXvY3s-5Az&YL0{EBxm`GX@GnGPU}nV2IQ7_%#1^L2ZLNd1o=hQ?W6GF739ShA~@* zC!PMuG5*q;!(UId-b=hH-yif&8iPJJ>X?m{VAFX^8q3G}Zw!9JKgh+ZU`|QB8}z+} z>3`z{!M_$C9$SpKDmvzY3dV)#{9%%+Q6y6;Fi8vTxm$xiU-c>4qny1!sgIo|p@b(+ z(<)6wQPS^VZa$eTxd08RC2^#AQv&2Y+!@7%$mQXe5`tfT_|(xyAqR`x)3E)V*0FI%~nQ9&AT5&0+ zBYA9Nu)KRyuVSb(Br0}X$UWChXvW)0TFfW5K4#mivM6){Q*~}k&Z)gT%+_5-)l!$j zM^rJ21VqjAw(^!az4o3$Seo}3ON~3$F;Yc-#DmK?{E2@olKlkmFN`UjP?5!LcC^0m z#njTr`3pnOdsnCOwq;>2Eb5228zO>0VDOJjH12~>24ZvyA_UMog#O_oo&)c%8{uJj z3Y`CpY5;4A|FByLfB&&kDQ)6EtR>yz{wLF}jPbVr;NO3%Dnh&vRXmg%VB=GEF)Sqj0?lN3>TUo5TBBk!x2#o|zaF{&QZu`je#4{V(MT>2SsaY<)Tl z_=xx~2b=TXYf78@U*B{}xc_yheBk!+Uv_tQcQ*IG){)}lzw8_w?3VX7@n63F6rlgl zBK=p=fB9fb|kC{%XegQLNjj2Pp|BIaK7wo&6 z{2~N%&#>c~*4*(Ulzd86Bpq-6Wk;e}7~Dts^Dae|xZCd*zM&_tmEW{eKqezuNvQN9cd! z|6NzQh5d)Y6G5$2tw4q_;;6>9ETkHOynt^9x~J1{8?3898&F7XgBM^QWKyjfZhsg7 ztA<-T00SB&A)5ptLz^aA9{Tgj%hCu#xU_E{iK$?<`~gebNlsL9eK8GKUm2{dFDEG& zIx#%`cV}57ko{)g>>Efi@tkyVfLGe(7>8w6wbIJxo?WACzt$&=R8R%ehBu~_?5z1heQ8p%rs zAeILB;1tx{L6-y>A9AZM+1g4qwMM%x_-LR8K=JA|sn$rCgUTZZ;!;&M6YedO<81X& z?06q3e2>Xm$pJX4jw+Z{_C>4CdUt3$8wP^esj}|hr1-svm!J9T5415_n$`AbTKm> zKB6yj{$~&UKO6bKhP3hje$#2`_n+>F4}66Gcc;9&@&B$PCHQ}T+%NsOyE*^!^`{v7 z|Bj{qYWu&k6aM}aZa{40|2op<{Lfc;{^vFyU7D+3UJ961@{&fQd(s(XALcwG8hJVk zRxNB^e2yIbftTiDMt&wG@30QMGMuq(&F6HSTMC&t?ek@T67$I!p)fZ;(}Aknd_3v1 zoQ?XdXO`jw%3sSq{@T+L`H!v527?)J`Sl6qkwlXL5~P~1xE$7`$WO!frWyUF)txZM}@`Mizo2@ zJ;@X@_@{sJq(A7jI#08_fFCugl4HeQz0NS_Y>&>sWWlT5JRr)D-E!w4l^A)j1E65t zP>jCv>h)5J_RZ~zH0J7J=PgHaj5NuZshtWyF7+WK!D0U)>Q>$?9&ZhU<-F&Z(U$>~Ywd#LWU3%r#2Iurar8Kj~iz)ulr;Y`G zM9^ag-@rc{0{lTR;m;IGG~h3b^ACRwEM`-EI=zM;{|Q_`R%1ZJ4}=)&v%Uaw(N<6o zQ&*SIpsBMUfb0#Bi^(cx*+$ zlf`t^5x~v1!C!mZAb8Ig`VrPBw3=l;X?EZaAHg~s;`c!vX1|h{+N!HOEmKrFceMk}4!X^5m7Ie~KILEkli#L8b2-aRXXbcua^izI+)se7vR))_#o;pp zvNmL?nK9vU>~h#wcj3+#uK0qB>9UA^;)^cJ&&w@4FsBNc&5pn(ZPF%f()!c?2VrD5 IS^x?G07$JU<^TWy From cb5c4d2a1573bfe1a654dbf2f813b6fba91bb482 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 6 Feb 2025 14:02:07 +0530 Subject: [PATCH 73/87] more tests --- .../cmd/importDataFileTaskPicker_test.go | 245 ++++++++++++++++++ 1 file changed, 245 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 5554891626..a8ae620d5e 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -235,6 +235,113 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) assert.Error(t, err) } +func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableAdheresToMaxTasksInProgress(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + // multiple tasks for same table. + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded", 1) + testutils.FatalIfError(t, err) + _, shardedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.sharded", 2) + testutils.FatalIfError(t, err) + _, colocatedTask1, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated", 3) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file4", ldataDir, "public.colocated", 3) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + shardedTask1, + shardedTask2, + colocatedTask1, + colocatedTask2, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + }, + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + shardedTask2.TableNameTup, + }, + } + + picker, err := NewColocatedAwareRandomTaskPicker(2, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // initially because maxInprogressTasks is 2, we will get 2 different tasks + pickedTask1, err := picker.NextTask() + assert.NoError(t, err) + pickedTask2, err := picker.NextTask() + assert.NoError(t, err) + assert.NotEqual(t, pickedTask1, pickedTask2) + + // no matter how many times we call NextTask therefater, + // it should return either pickedTask1 or pickedTask2 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == pickedTask1 || task == pickedTask2, "task: %v, pickedTask1: %v, pickedTask2: %v", task, pickedTask1, pickedTask2) + } + + // mark task1 as done + err = picker.MarkTaskAsDone(pickedTask1) + assert.NoError(t, err) + + // keep picking tasks until we get a task that is not pickedTask2 + var pickedTask3 *ImportFileTask + for { + task, err := picker.NextTask() + if err != nil { + break + } + if task != pickedTask2 && task != pickedTask1 { + pickedTask3 = task + break + } + } + + // now, next task should be either pickedTask2 or pickedTask3 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == pickedTask2 || task == pickedTask3, "task: %v, pickedTask2: %v, pickedTask3: %v", task, pickedTask2, pickedTask3) + } + + // mark task2, task3 as done + err = picker.MarkTaskAsDone(pickedTask2) + assert.NoError(t, err) + err = picker.MarkTaskAsDone(pickedTask3) + assert.NoError(t, err) + + // now, next task should be pickedTask4, which is not one of the previous tasks + pickedTask4, err := picker.NextTask() + assert.NoError(t, err) + assert.NotContains(t, []*ImportFileTask{pickedTask1, pickedTask2, pickedTask3}, pickedTask4) + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == pickedTask4, "task: %v, pickedTask4: %v", task, pickedTask2) + } + + // mark task4 as done + err = picker.MarkTaskAsDone(pickedTask4) + assert.NoError(t, err) + + // now, there should be no more tasks + assert.False(t, picker.HasMoreTasks()) + _, err = picker.NextTask() + assert.Error(t, err) +} + func TestColocatedAwareRandomTaskPickerSingleTask(t *testing.T) { ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) testutils.FatalIfError(t, err) @@ -352,6 +459,75 @@ func TestColocatedAwareRandomTaskPickerTasksEqualToMaxTasksInProgress(t *testing assert.Error(t, err) } +func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableTasksEqualToMaxTasksInProgress(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask1, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated1", 2) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated1", 3) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + shardedTask1, + colocatedTask1, + colocatedTask2, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + }, + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + }, + } + + // 3 tasks, 3 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(3, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask therefater, + // it should return one of the tasks + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == shardedTask1 || task == colocatedTask1 || task == colocatedTask2, "task: %v, expected tasks = %v", task, tasks) + } + + // mark task1 as done + err = picker.MarkTaskAsDone(shardedTask1) + assert.NoError(t, err) + + // now, next task should be either colocatedTask1 or colocatedTask2 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == colocatedTask1 || task == colocatedTask2, "task: %v, colocatedTask1: %v, colocatedTask2: %v", task, colocatedTask1, colocatedTask2) + } + + // mark colocatedTask1, colocatedTask2 as done + err = picker.MarkTaskAsDone(colocatedTask1) + assert.NoError(t, err) + err = picker.MarkTaskAsDone(colocatedTask2) + assert.NoError(t, err) + + // now, there should be no more tasks + assert.False(t, picker.HasMoreTasks()) + _, err = picker.NextTask() + assert.Error(t, err) +} + func TestColocatedAwareRandomTaskPickerTasksLessThanMaxTasksInProgress(t *testing.T) { ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) testutils.FatalIfError(t, err) @@ -421,6 +597,75 @@ func TestColocatedAwareRandomTaskPickerTasksLessThanMaxTasksInProgress(t *testin assert.Error(t, err) } +func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableTasksLessThanMaxTasksInProgress(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask1, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated1", 2) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated1", 3) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + shardedTask1, + colocatedTask1, + colocatedTask2, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + }, + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + }, + } + + // 3 tasks, 10 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask therefater, + // it should return one of the tasks + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == shardedTask1 || task == colocatedTask1 || task == colocatedTask2, "task: %v, expected tasks = %v", task, tasks) + } + + // mark task1 as done + err = picker.MarkTaskAsDone(shardedTask1) + assert.NoError(t, err) + + // now, next task should be either colocatedTask1 or colocatedTask2 + for i := 0; i < 100; i++ { + task, err := picker.NextTask() + assert.NoError(t, err) + assert.Truef(t, task == colocatedTask1 || task == colocatedTask2, "task: %v, colocatedTask1: %v, colocatedTask2: %v", task, colocatedTask1, colocatedTask2) + } + + // mark colocatedTask1, colocatedTask2 as done + err = picker.MarkTaskAsDone(colocatedTask1) + assert.NoError(t, err) + err = picker.MarkTaskAsDone(colocatedTask2) + assert.NoError(t, err) + + // now, there should be no more tasks + assert.False(t, picker.HasMoreTasks()) + _, err = picker.NextTask() + assert.Error(t, err) +} + func TestColocatedAwareRandomTaskPickerAllShardedTasks(t *testing.T) { ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) testutils.FatalIfError(t, err) From 93eb55a437c19b66bd5ae9b3be31da230a42c1cd Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 6 Feb 2025 15:04:17 +0530 Subject: [PATCH 74/87] use colocatedAwareRandomTaskPicker --- yb-voyager/cmd/importData.go | 22 +++++++++++++++++----- yb-voyager/src/tgtdb/yugabytedb.go | 14 ++++++++++++++ 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index 5d821cfc54..3d540bafda 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -576,6 +576,7 @@ func importData(importFileTasks []*ImportFileTask) { utils.PrintAndLog("Tables to import: %v", importFileTasksToTableNames(pendingTasks)) prepareTableToColumns(pendingTasks) //prepare the tableToColumns map poolSize := tconf.Parallelism * 2 + maxTasksInProgress := tconf.Parallelism if tconf.EnableYBAdaptiveParallelism { // in case of adaptive parallelism, we need to use maxParalllelism * 2 yb, ok := tdb.(*tgtdb.TargetYugabyteDB) @@ -593,7 +594,7 @@ func importData(importFileTasks []*ImportFileTask) { useTaskPicker := utils.GetEnvAsBool("USE_TASK_PICKER_FOR_IMPORT", true) if useTaskPicker { - err := importTasksViaTaskPicker(pendingTasks, state, progressReporter, poolSize) + err := importTasksViaTaskPicker(pendingTasks, state, progressReporter, poolSize, maxTasksInProgress) if err != nil { utils.ErrExit("Failed to import tasks via task picker: %s", err) } @@ -699,16 +700,27 @@ func importData(importFileTasks []*ImportFileTask) { } -func importTasksViaTaskPicker(pendingTasks []*ImportFileTask, state *ImportDataState, progressReporter *ImportDataProgressReporter, poolSize int) error { +func importTasksViaTaskPicker(pendingTasks []*ImportFileTask, state *ImportDataState, progressReporter *ImportDataProgressReporter, poolSize int, maxTasksInProgress int) error { // The code can produce `poolSize` number of batches at a time. But, it can consume only // `parallelism` number of batches at a time. batchImportPool = pool.New().WithMaxGoroutines(poolSize) log.Infof("created batch import pool of size: %d", poolSize) - taskPicker, err := NewSequentialTaskPicker(pendingTasks, state) - if err != nil { - return fmt.Errorf("create task picker: %w", err) + var taskPicker FileTaskPicker + var err error + if importerRole == TARGET_DB_IMPORTER_ROLE || importerRole == IMPORT_FILE_ROLE { + yb, ok := tdb.(*tgtdb.TargetYugabyteDB) + if !ok { + return fmt.Errorf("expected tdb to be of type TargetYugabyteDB, got: %T", tdb) + } + taskPicker, err = NewColocatedAwareRandomTaskPicker(maxTasksInProgress, pendingTasks, state, yb) + } else { + taskPicker, err = NewSequentialTaskPicker(pendingTasks, state) + if err != nil { + return fmt.Errorf("create task picker: %w", err) + } } + taskImporters := map[int]*FileTaskImporter{} for taskPicker.HasMoreTasks() { diff --git a/yb-voyager/src/tgtdb/yugabytedb.go b/yb-voyager/src/tgtdb/yugabytedb.go index 7c4e179f85..181386c9e7 100644 --- a/yb-voyager/src/tgtdb/yugabytedb.go +++ b/yb-voyager/src/tgtdb/yugabytedb.go @@ -1228,6 +1228,20 @@ func (yb *TargetYugabyteDB) isQueryResultNonEmpty(query string) bool { return rows.Next() } +func (yb *TargetYugabyteDB) IsDBColocated() (bool, error) { + query := "select yb_is_database_colocated()" + var isColocated bool + err := yb.QueryRow(query).Scan(&isColocated) + return isColocated, err +} + +func (yb *TargetYugabyteDB) IsTableColocated(tableName sqlname.NameTuple) (bool, error) { + query := fmt.Sprintf("select is_colocated from yb_table_properties('%s'::regclass)", tableName.ForUserQuery()) + var isTableColocated bool + err := yb.QueryRow(query).Scan(&isTableColocated) + return isTableColocated, err +} + func (yb *TargetYugabyteDB) IsAdaptiveParallelismSupported() bool { query := "SELECT * FROM pg_proc WHERE proname='yb_servers_metrics'" return yb.isQueryResultNonEmpty(query) From 957ef0c13a32657a52b21c75180eb71994f3b109 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 6 Feb 2025 19:53:46 +0530 Subject: [PATCH 75/87] return error --- yb-voyager/cmd/importData.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index 3d540bafda..51cc0d43f3 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -714,10 +714,13 @@ func importTasksViaTaskPicker(pendingTasks []*ImportFileTask, state *ImportDataS return fmt.Errorf("expected tdb to be of type TargetYugabyteDB, got: %T", tdb) } taskPicker, err = NewColocatedAwareRandomTaskPicker(maxTasksInProgress, pendingTasks, state, yb) + if err != nil { + return fmt.Errorf("create colocated aware randmo task picker: %w", err) + } } else { taskPicker, err = NewSequentialTaskPicker(pendingTasks, state) if err != nil { - return fmt.Errorf("create task picker: %w", err) + return fmt.Errorf("create sequential task picker: %w", err) } } From df3cded56d655466657ab3556eee4900788fb87c Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Thu, 6 Feb 2025 22:08:21 +0530 Subject: [PATCH 76/87] fix for resuming --- yb-voyager/cmd/importDataFileTaskPicker.go | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 6b773fb5c9..9b791c4751 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -169,7 +169,19 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi case FILE_IMPORT_COMPLETED: doneTasks = append(doneTasks, task) case FILE_IMPORT_IN_PROGRESS: - inProgressTasks = append(inProgressTasks, task) + if len(inProgressTasks) < maxTasksInProgress { + inProgressTasks = append(inProgressTasks, task) + } else { + // put into the table wise pending tasks. + var tablePendingTasks []*ImportFileTask + var ok bool + tablePendingTasks, ok = tableWisePendingTasks.Get(tableName) + if !ok { + tablePendingTasks = []*ImportFileTask{} + } + tablePendingTasks = append(tablePendingTasks, task) + tableWisePendingTasks.Put(tableName, tablePendingTasks) + } case FILE_IMPORT_NOT_STARTED: // put into the table wise pending tasks. var tablePendingTasks []*ImportFileTask From 0fd4447a095c609b63e75087bcc8cc27ee799ec3 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 12 Feb 2025 12:51:14 +0530 Subject: [PATCH 77/87] resolve merge --- .../data/gather-assessment-metadata.tar.gz | Bin 10211 -> 10211 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/yb-voyager/src/srcdb/data/gather-assessment-metadata.tar.gz b/yb-voyager/src/srcdb/data/gather-assessment-metadata.tar.gz index c6d9f40c96d01211937b12744e9309d4d0afa18c..bc8d7b6a1643ccd3e8158ea0ffc2538a38f031d8 100644 GIT binary patch delta 16 XcmaFt|Ja{hzMF&Lf%lq??03`wI->@# delta 16 XcmaFt|Ja{hzMF%A*>3Sh_B-kTG#v%K From 2ba9d5fb36898f9aaca6707893cb39d50ecc6349 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 12 Feb 2025 13:11:51 +0530 Subject: [PATCH 78/87] resolve merge conflicts --- yb-voyager/cmd/importDataFileTaskPicker.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 0cda834164..52a23bb33e 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -33,7 +33,7 @@ const ( ) type FileTaskPicker interface { - NextTask() (*ImportFileTask, error) + Pick() (*ImportFileTask, error) MarkTaskAsDone(task *ImportFileTask) error HasMoreTasks() bool WaitForTasksBatchesTobeImported() @@ -232,7 +232,7 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi }, nil } -func (c *ColocatedAwareRandomTaskPicker) NextTask() (*ImportFileTask, error) { +func (c *ColocatedAwareRandomTaskPicker) Pick() (*ImportFileTask, error) { if !c.HasMoreTasks() { return nil, fmt.Errorf("no more tasks") } @@ -369,3 +369,8 @@ func (c *ColocatedAwareRandomTaskPicker) HasMoreTasks() bool { return pendingTasks } + +func (c *ColocatedAwareRandomTaskPicker) WaitForTasksBatchesTobeImported() { + // no wait + return +} From 0e21028bf5b5e49a95ae18694cee69e5da19390d Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Wed, 12 Feb 2025 15:05:13 +0530 Subject: [PATCH 79/87] fix test --- .../cmd/importDataFileTaskPicker_test.go | 96 +++++++++---------- 1 file changed, 48 insertions(+), 48 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 548a7dbe23..75ad4c9c37 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -73,7 +73,7 @@ func TestSequentialTaskPickerBasic(t *testing.T) { assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call NextTask, it should return the same task (first task) + // no matter how many times we call Pick, it should return the same task (first task) for i := 0; i < 10; i++ { task, err := picker.Pick() assert.NoError(t, err) @@ -107,7 +107,7 @@ func TestSequentialTaskPickerMarkTaskDone(t *testing.T) { assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call NextTask, it should return the same task (first task) + // no matter how many times we call Pick, it should return the same task (first task) for i := 0; i < 10; i++ { task, err := picker.Pick() assert.NoError(t, err) @@ -176,16 +176,16 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) assert.True(t, picker.HasMoreTasks()) // initially because maxInprogressTasks is 2, we will get 2 different tasks - pickedTask1, err := picker.NextTask() + pickedTask1, err := picker.Pick() assert.NoError(t, err) - pickedTask2, err := picker.NextTask() + pickedTask2, err := picker.Pick() assert.NoError(t, err) assert.NotEqual(t, pickedTask1, pickedTask2) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return either pickedTask1 or pickedTask2 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == pickedTask1 || task == pickedTask2, "task: %v, pickedTask1: %v, pickedTask2: %v", task, pickedTask1, pickedTask2) } @@ -197,7 +197,7 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) // keep picking tasks until we get a task that is not pickedTask2 var pickedTask3 *ImportFileTask for { - task, err := picker.NextTask() + task, err := picker.Pick() if err != nil { break } @@ -209,7 +209,7 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) // now, next task should be either pickedTask2 or pickedTask3 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == pickedTask2 || task == pickedTask3, "task: %v, pickedTask2: %v, pickedTask3: %v", task, pickedTask2, pickedTask3) } @@ -220,7 +220,7 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) // now, next task should be pickedTask2 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == pickedTask2, "task: %v, pickedTask2: %v", task, pickedTask2) } @@ -231,7 +231,7 @@ func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) // now, there should be no more tasks assert.False(t, picker.HasMoreTasks()) - _, err = picker.NextTask() + _, err = picker.Pick() assert.Error(t, err) } @@ -278,16 +278,16 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableAdheresToMaxTasksInP assert.True(t, picker.HasMoreTasks()) // initially because maxInprogressTasks is 2, we will get 2 different tasks - pickedTask1, err := picker.NextTask() + pickedTask1, err := picker.Pick() assert.NoError(t, err) - pickedTask2, err := picker.NextTask() + pickedTask2, err := picker.Pick() assert.NoError(t, err) assert.NotEqual(t, pickedTask1, pickedTask2) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return either pickedTask1 or pickedTask2 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == pickedTask1 || task == pickedTask2, "task: %v, pickedTask1: %v, pickedTask2: %v", task, pickedTask1, pickedTask2) } @@ -299,7 +299,7 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableAdheresToMaxTasksInP // keep picking tasks until we get a task that is not pickedTask2 var pickedTask3 *ImportFileTask for { - task, err := picker.NextTask() + task, err := picker.Pick() if err != nil { break } @@ -311,7 +311,7 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableAdheresToMaxTasksInP // now, next task should be either pickedTask2 or pickedTask3 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == pickedTask2 || task == pickedTask3, "task: %v, pickedTask2: %v, pickedTask3: %v", task, pickedTask2, pickedTask3) } @@ -323,11 +323,11 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableAdheresToMaxTasksInP assert.NoError(t, err) // now, next task should be pickedTask4, which is not one of the previous tasks - pickedTask4, err := picker.NextTask() + pickedTask4, err := picker.Pick() assert.NoError(t, err) assert.NotContains(t, []*ImportFileTask{pickedTask1, pickedTask2, pickedTask3}, pickedTask4) for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == pickedTask4, "task: %v, pickedTask4: %v", task, pickedTask2) } @@ -338,7 +338,7 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableAdheresToMaxTasksInP // now, there should be no more tasks assert.False(t, picker.HasMoreTasks()) - _, err = picker.NextTask() + _, err = picker.Pick() assert.Error(t, err) } @@ -369,13 +369,13 @@ func TestColocatedAwareRandomTaskPickerSingleTask(t *testing.T) { testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - pickedTask1, err := picker.NextTask() + pickedTask1, err := picker.Pick() assert.NoError(t, err) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return either pickedTask1 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == pickedTask1, "task: %v, pickedTask1: %v", task, pickedTask1) } @@ -386,7 +386,7 @@ func TestColocatedAwareRandomTaskPickerSingleTask(t *testing.T) { // now, there should be no more tasks assert.False(t, picker.HasMoreTasks()) - _, err = picker.NextTask() + _, err = picker.Pick() assert.Error(t, err) } @@ -428,10 +428,10 @@ func TestColocatedAwareRandomTaskPickerTasksEqualToMaxTasksInProgress(t *testing testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return one of the tasks for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == shardedTask1 || task == colocatedTask1 || task == colocatedTask2, "task: %v, expected tasks = %v", task, tasks) } @@ -442,7 +442,7 @@ func TestColocatedAwareRandomTaskPickerTasksEqualToMaxTasksInProgress(t *testing // now, next task should be either colocatedTask1 or colocatedTask2 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == colocatedTask1 || task == colocatedTask2, "task: %v, colocatedTask1: %v, colocatedTask2: %v", task, colocatedTask1, colocatedTask2) } @@ -455,7 +455,7 @@ func TestColocatedAwareRandomTaskPickerTasksEqualToMaxTasksInProgress(t *testing // now, there should be no more tasks assert.False(t, picker.HasMoreTasks()) - _, err = picker.NextTask() + _, err = picker.Pick() assert.Error(t, err) } @@ -497,10 +497,10 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableTasksEqualToMaxTasks testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return one of the tasks for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == shardedTask1 || task == colocatedTask1 || task == colocatedTask2, "task: %v, expected tasks = %v", task, tasks) } @@ -511,7 +511,7 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableTasksEqualToMaxTasks // now, next task should be either colocatedTask1 or colocatedTask2 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == colocatedTask1 || task == colocatedTask2, "task: %v, colocatedTask1: %v, colocatedTask2: %v", task, colocatedTask1, colocatedTask2) } @@ -524,7 +524,7 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableTasksEqualToMaxTasks // now, there should be no more tasks assert.False(t, picker.HasMoreTasks()) - _, err = picker.NextTask() + _, err = picker.Pick() assert.Error(t, err) } @@ -566,10 +566,10 @@ func TestColocatedAwareRandomTaskPickerTasksLessThanMaxTasksInProgress(t *testin testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return one of the tasks for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == shardedTask1 || task == colocatedTask1 || task == colocatedTask2, "task: %v, expected tasks = %v", task, tasks) } @@ -580,7 +580,7 @@ func TestColocatedAwareRandomTaskPickerTasksLessThanMaxTasksInProgress(t *testin // now, next task should be either colocatedTask1 or colocatedTask2 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == colocatedTask1 || task == colocatedTask2, "task: %v, colocatedTask1: %v, colocatedTask2: %v", task, colocatedTask1, colocatedTask2) } @@ -593,7 +593,7 @@ func TestColocatedAwareRandomTaskPickerTasksLessThanMaxTasksInProgress(t *testin // now, there should be no more tasks assert.False(t, picker.HasMoreTasks()) - _, err = picker.NextTask() + _, err = picker.Pick() assert.Error(t, err) } @@ -635,10 +635,10 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableTasksLessThanMaxTask testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return one of the tasks for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == shardedTask1 || task == colocatedTask1 || task == colocatedTask2, "task: %v, expected tasks = %v", task, tasks) } @@ -649,7 +649,7 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableTasksLessThanMaxTask // now, next task should be either colocatedTask1 or colocatedTask2 for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == colocatedTask1 || task == colocatedTask2, "task: %v, colocatedTask1: %v, colocatedTask2: %v", task, colocatedTask1, colocatedTask2) } @@ -662,7 +662,7 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableTasksLessThanMaxTask // now, there should be no more tasks assert.False(t, picker.HasMoreTasks()) - _, err = picker.NextTask() + _, err = picker.Pick() assert.Error(t, err) } @@ -702,10 +702,10 @@ func TestColocatedAwareRandomTaskPickerAllShardedTasks(t *testing.T) { testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return one of the tasks for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == shardedTask1 || task == shardedTask2 || task == shardedTask3, "task: %v, expected tasks = %v", task, tasks) } @@ -752,7 +752,7 @@ func TestColocatedAwareRandomTaskPickerAllShardedTasksChooser(t *testing.T) { assertTableChooserPicksShardedAndColocatedAsExpected(t, picker.tableChooser, dummyYb.colocatedTables, dummyYb.shardedTables) // now pick one task. After picking that task, the chooser should have updated probabilities - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) updatedPendingColocatedTables := lo.Filter(dummyYb.colocatedTables, func(t sqlname.NameTuple, _ int) bool { @@ -866,10 +866,10 @@ func TestColocatedAwareRandomTaskPickerAllColocatedTasks(t *testing.T) { testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return one of the tasks for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3, "task: %v, expected tasks = %v", task, tasks) } @@ -916,7 +916,7 @@ func TestColocatedAwareRandomTaskPickerAllColocatedTasksChooser(t *testing.T) { assertTableChooserPicksShardedAndColocatedAsExpected(t, picker.tableChooser, dummyYb.colocatedTables, dummyYb.shardedTables) // now pick one task. After picking that task, the chooser should have updated probabilities - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) updatedPendingColocatedTables := lo.Filter(dummyYb.colocatedTables, func(t sqlname.NameTuple, _ int) bool { @@ -975,10 +975,10 @@ func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasks(t *testing.T) { testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call NextTask therefater, + // no matter how many times we call Pick therefater, // it should return one of the tasks for i := 0; i < 100; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3 || task == shardedTask1 || task == shardedTask2, "task: %v, expected tasks = %v", task, tasks) } @@ -1041,7 +1041,7 @@ func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasksChooser(t *testin // now pick tasks one by one. After picking each task, the chooser should have updated probabilities for i := 0; i < 4; i++ { - task, err := picker.NextTask() + task, err := picker.Pick() assert.NoError(t, err) updatedPendingColocatedTables = lo.Filter(updatedPendingColocatedTables, func(t sqlname.NameTuple, _ int) bool { From 59edf8ef527b34091ffcb08cb5fd1acd04ee83b6 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 17 Feb 2025 10:40:36 +0530 Subject: [PATCH 80/87] logs --- yb-voyager/cmd/importData.go | 3 +++ yb-voyager/cmd/importDataFileTaskPicker.go | 12 +++++++++--- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/yb-voyager/cmd/importData.go b/yb-voyager/cmd/importData.go index 22945c1e10..faf1f7c411 100644 --- a/yb-voyager/cmd/importData.go +++ b/yb-voyager/cmd/importData.go @@ -740,6 +740,7 @@ func importTasksViaTaskPicker(pendingTasks []*ImportFileTask, state *ImportDataS if err != nil { return fmt.Errorf("get next task: %w", err) } + log.Infof("Picked task for import: %s", task) var taskImporter *FileTaskImporter var ok bool taskImporter, ok = taskImporters[task.ID] @@ -756,6 +757,7 @@ func importTasksViaTaskPicker(pendingTasks []*ImportFileTask, state *ImportDataS // All batches for this task have been submitted. // task could have been completed (all batches imported) OR still in progress // in case task is done, we should inform task picker so that we stop picking that task. + log.Infof("All batches submitted for task: %s", task) taskDone, err := taskImporter.AllBatchesImported() if err != nil { return fmt.Errorf("check if all batches are imported: task: %v err :%w", task, err) @@ -766,6 +768,7 @@ func importTasksViaTaskPicker(pendingTasks []*ImportFileTask, state *ImportDataS if err != nil { return fmt.Errorf("mark task as done: task: %v, err: %w", task, err) } + log.Infof("Import of task done: %s", task) continue } else { // some batches are still in progress, wait for them to complete as decided by the picker. diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 52a23bb33e..8cd721ecad 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -20,8 +20,10 @@ import ( "time" + "github.com/davecgh/go-spew/spew" "github.com/mroth/weightedrand/v2" "github.com/samber/lo" + log "github.com/sirupsen/logrus" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" "golang.org/x/exp/rand" @@ -157,7 +159,6 @@ type YbTargetDBColocatedChecker interface { } func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFileTask, state *ImportDataState, yb YbTargetDBColocatedChecker) (*ColocatedAwareRandomTaskPicker, error) { - // var pendingTasks []*ImportFileTask var doneTasks []*ImportFileTask var inProgressTasks []*ImportFileTask tableWisePendingTasks := utils.NewStructMap[sqlname.NameTuple, []*ImportFileTask]() @@ -223,13 +224,16 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi } } - return &ColocatedAwareRandomTaskPicker{ + picker := &ColocatedAwareRandomTaskPicker{ doneTasks: doneTasks, inProgressTasks: inProgressTasks, maxTasksInProgress: maxTasksInProgress, tableWisePendingTasks: tableWisePendingTasks, tableTypes: tableTypes, - }, nil + } + + log.Infof("ColocatedAwareRandomTaskPicker initialized with params:%v", spew.Sdump(picker)) + return picker, nil } func (c *ColocatedAwareRandomTaskPicker) Pick() (*ImportFileTask, error) { @@ -292,6 +296,7 @@ func (c *ColocatedAwareRandomTaskPicker) PickTaskFromPendingTasks() (*ImportFile c.tableWisePendingTasks.Put(tablePick, tablePendingTasks) } c.inProgressTasks = append(c.inProgressTasks, pickedTask) + log.Infof("Picked task: %v. In-Progress tasks:%v", pickedTask, c.inProgressTasks) return pickedTask, nil } @@ -347,6 +352,7 @@ func (c *ColocatedAwareRandomTaskPicker) MarkTaskAsDone(task *ImportFileTask) er if t.ID == task.ID { c.inProgressTasks = append(c.inProgressTasks[:i], c.inProgressTasks[i+1:]...) c.doneTasks = append(c.doneTasks, task) + log.Infof("Marked task as done: %v. In-Progress tasks:%v", t, c.inProgressTasks) return nil } } From fcf19b2bd7426c50e0d1c863e73a3b704d5cecf7 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 17 Feb 2025 10:45:17 +0530 Subject: [PATCH 81/87] resumable test --- .../cmd/importDataFileTaskPicker_test.go | 97 ++++++++++++++++--- 1 file changed, 83 insertions(+), 14 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 75ad4c9c37..1de18ae836 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -1055,17 +1055,86 @@ func TestColocatedAwareRandomTaskPickerMixShardedColocatedTasksChooser(t *testin } } -/* -singleTask -maxTasksInProgressEqualToTotalTasks -maxTasksInProgressGreaterThanTotalTasks -MixOfColocatedAndShardedTables - ensure all are getting picked with proper weights - test that weights change when tasks are marked as done -AllColocatedTables - ensure all are getting picked with proper weights - test that weights change when tasks are marked as done -AllShardedTables - ensure all are getting picked with proper weights - test that weights change when tasks are marked as done - -all of the above cases for -multipleTasksPerTable (importDataFileCase) -*/ +func TestColocatedAwareRandomTaskPickerResumable(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, colocatedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.colocated1", 1) + testutils.FatalIfError(t, err) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.colocated2", 2) + testutils.FatalIfError(t, err) + _, colocatedTask3, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated3", 3) + testutils.FatalIfError(t, err) + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) + testutils.FatalIfError(t, err) + _, shardedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.sharded2", 2) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + colocatedTask1, + colocatedTask2, + colocatedTask3, + shardedTask1, + shardedTask2, + } + dummyYb := &dummyYb{ + colocatedTables: []sqlname.NameTuple{ + colocatedTask1.TableNameTup, + colocatedTask2.TableNameTup, + colocatedTask3.TableNameTup, + }, + shardedTables: []sqlname.NameTuple{ + shardedTask1.TableNameTup, + shardedTask2.TableNameTup, + }, + } + + // 5 tasks, 10 max tasks in progress + picker, err := NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call Pick therefater, + // it should return one of the tasks + for i := 0; i < 100; i++ { + task, err := picker.Pick() + assert.NoError(t, err) + assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3 || task == shardedTask1 || task == shardedTask2, "task: %v, expected tasks = %v", task, tasks) + } + assert.Equal(t, 5, len(picker.inProgressTasks)) + + // simulate restart with a smaller no. of max tasks in progress + picker, err = NewColocatedAwareRandomTaskPicker(2, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + // it should return one of the tasks + for i := 0; i < 100; i++ { + task, err := picker.Pick() + assert.NoError(t, err) + assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3 || task == shardedTask1 || task == shardedTask2, "task: %v, expected tasks = %v", task, tasks) + } + + // only two shoudl be inprogress even though previously 5 were in progress. + assert.Equal(t, 2, len(picker.inProgressTasks)) + + // simulate restart with a larger no. of max tasks in progress + picker, err = NewColocatedAwareRandomTaskPicker(20, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + // it should return one of the tasks + for i := 0; i < 100; i++ { + task, err := picker.Pick() + assert.NoError(t, err) + assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3 || task == shardedTask1 || task == shardedTask2, "task: %v, expected tasks = %v", task, tasks) + } + + // all 5 should be inprogress + assert.Equal(t, 5, len(picker.inProgressTasks)) +} From aad59d411165aa227a421072d3610a17991baefc Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 17 Feb 2025 11:08:04 +0530 Subject: [PATCH 82/87] resumable test --- .../cmd/importDataFileTaskPicker_test.go | 66 +++++++++++-------- 1 file changed, 38 insertions(+), 28 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index 1de18ae836..aa5d1c6b45 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -25,11 +25,13 @@ import ( "github.com/mroth/weightedrand/v2" "github.com/samber/lo" "github.com/stretchr/testify/assert" + "github.com/yugabyte/yb-voyager/yb-voyager/src/tgtdb" "github.com/yugabyte/yb-voyager/yb-voyager/src/utils/sqlname" testutils "github.com/yugabyte/yb-voyager/yb-voyager/test/utils" ) type dummyYb struct { + tgtdb.TargetYugabyteDB colocatedTables []sqlname.NameTuple shardedTables []sqlname.NameTuple } @@ -47,6 +49,10 @@ func (d *dummyYb) IsDBColocated() (bool, error) { return len(d.colocatedTables) > 0, nil } +func (d *dummyYb) ImportBatch(batch tgtdb.Batch, args *tgtdb.ImportBatchArgs, exportDir string, tableSchema map[string]map[string]string) (int64, error) { + return 1, nil +} + func TestSequentialTaskPickerBasic(t *testing.T) { ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) testutils.FatalIfError(t, err) @@ -1101,40 +1107,44 @@ func TestColocatedAwareRandomTaskPickerResumable(t *testing.T) { testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // no matter how many times we call Pick therefater, - // it should return one of the tasks - for i := 0; i < 100; i++ { - task, err := picker.Pick() - assert.NoError(t, err) - assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3 || task == shardedTask1 || task == shardedTask2, "task: %v, expected tasks = %v", task, tasks) - } - assert.Equal(t, 5, len(picker.inProgressTasks)) + // pick 3 tasks and start the process. + task1, err := picker.Pick() + assert.NoError(t, err) + fbp1, err := NewFileBatchProducer(task1, state) + batch1, err := fbp1.NextBatch() + assert.NoError(t, err) + batch1.MarkInProgress() - // simulate restart with a smaller no. of max tasks in progress - picker, err = NewColocatedAwareRandomTaskPicker(2, tasks, state, dummyYb) + task2, err := picker.Pick() + assert.NoError(t, err) + fbp2, err := NewFileBatchProducer(task2, state) + batch2, err := fbp2.NextBatch() + assert.NoError(t, err) + batch2.MarkInProgress() + + task3, err := picker.Pick() + assert.NoError(t, err) + fbp3, err := NewFileBatchProducer(task3, state) + batch3, err := fbp3.NextBatch() + assert.NoError(t, err) + batch3.MarkInProgress() + + // simulate restart. now, those 3 tasks should be in progress + picker, err = NewColocatedAwareRandomTaskPicker(10, tasks, state, dummyYb) testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // it should return one of the tasks - for i := 0; i < 100; i++ { - task, err := picker.Pick() - assert.NoError(t, err) - assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3 || task == shardedTask1 || task == shardedTask2, "task: %v, expected tasks = %v", task, tasks) - } - - // only two shoudl be inprogress even though previously 5 were in progress. - assert.Equal(t, 2, len(picker.inProgressTasks)) + assert.Equal(t, 3, len(picker.inProgressTasks)) - // simulate restart with a larger no. of max tasks in progress + // simulate restart with a larger no. of max tasks in progress picker, err = NewColocatedAwareRandomTaskPicker(20, tasks, state, dummyYb) testutils.FatalIfError(t, err) assert.True(t, picker.HasMoreTasks()) - // it should return one of the tasks - for i := 0; i < 100; i++ { - task, err := picker.Pick() - assert.NoError(t, err) - assert.Truef(t, task == colocatedTask1 || task == colocatedTask2 || task == colocatedTask3 || task == shardedTask1 || task == shardedTask2, "task: %v, expected tasks = %v", task, tasks) - } + assert.Equal(t, 3, len(picker.inProgressTasks)) - // all 5 should be inprogress - assert.Equal(t, 5, len(picker.inProgressTasks)) + // simulate restart with a smaller no. of max tasks in progress + picker, err = NewColocatedAwareRandomTaskPicker(2, tasks, state, dummyYb) + testutils.FatalIfError(t, err) + assert.True(t, picker.HasMoreTasks()) + // only two shoudl be inprogress even though previously 3 were in progress. + assert.Equal(t, 2, len(picker.inProgressTasks)) } From b6d968adb65a1525cb0720f80a6de61b0c5215f0 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 17 Feb 2025 11:17:55 +0530 Subject: [PATCH 83/87] more comments; initialize chooser early instead of lazy --- yb-voyager/cmd/importDataFileTaskPicker.go | 26 ++++++++++++++-------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 8cd721ecad..295e59ab8f 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -142,15 +142,19 @@ At any given time, only X distinct tables can be IN-PROGRESS. If X=4, after pic During this time, each of the in-progress tables will be picked with equal probability. */ type ColocatedAwareRandomTaskPicker struct { - // pendingTasks []*ImportFileTask - doneTasks []*ImportFileTask - inProgressTasks []*ImportFileTask + doneTasks []*ImportFileTask + // tasks which the picker has picked at least once, and are essentially in progress. + // the length of this list will be <= maxTasksInProgress + inProgressTasks []*ImportFileTask + maxTasksInProgress int + + // tasks which have not yet been picked even once. + // the tableChooser will be employed to pick a table from this list. tableWisePendingTasks *utils.StructMap[sqlname.NameTuple, []*ImportFileTask] - maxTasksInProgress int + tableChooser *weightedrand.Chooser[sqlname.NameTuple, int] tableTypes *utils.StructMap[sqlname.NameTuple, string] //colocated or sharded - tableChooser *weightedrand.Chooser[sqlname.NameTuple, int] } type YbTargetDBColocatedChecker interface { @@ -231,6 +235,10 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi tableWisePendingTasks: tableWisePendingTasks, tableTypes: tableTypes, } + err = picker.initializeChooser() + if err != nil { + return nil, fmt.Errorf("initializing chooser: %w", err) + } log.Infof("ColocatedAwareRandomTaskPicker initialized with params:%v", spew.Sdump(picker)) return picker, nil @@ -241,10 +249,6 @@ func (c *ColocatedAwareRandomTaskPicker) Pick() (*ImportFileTask, error) { return nil, fmt.Errorf("no more tasks") } - if c.tableChooser == nil { - c.initializeChooser() - } - // if we have already picked maxTasksInProgress tasks, pick a task from inProgressTasks if len(c.inProgressTasks) == c.maxTasksInProgress { return c.PickTaskFromInProgressTasks() @@ -273,6 +277,10 @@ func (c *ColocatedAwareRandomTaskPicker) PickTaskFromPendingTasks() (*ImportFile if len(c.tableWisePendingTasks.Keys()) == 0 { return nil, fmt.Errorf("no pending tasks to pick from") } + if c.tableChooser == nil { + return nil, fmt.Errorf("chooser not initialized") + } + tablePick := c.tableChooser.Pick() tablePendingTasks, ok := c.tableWisePendingTasks.Get(tablePick) if !ok { From ade6ef60bedafa1b3fac3905edd388ae9e07503d Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Mon, 17 Feb 2025 11:20:38 +0530 Subject: [PATCH 84/87] review comments --- yb-voyager/cmd/importDataFileTaskPicker.go | 32 ++++++++++------------ 1 file changed, 14 insertions(+), 18 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 295e59ab8f..97fc860fae 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -173,6 +173,18 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi return nil, fmt.Errorf("checking if db is colocated: %w", err) } + addToPendingTasks := func(t *ImportFileTask) { + // put into the table wise pending tasks. + var tablePendingTasks []*ImportFileTask + var ok bool + tablePendingTasks, ok = tableWisePendingTasks.Get(t.TableNameTup) + if !ok { + tablePendingTasks = []*ImportFileTask{} + } + tablePendingTasks = append(tablePendingTasks, t) + tableWisePendingTasks.Put(t.TableNameTup, tablePendingTasks) + } + for _, task := range tasks { tableName := task.TableNameTup @@ -203,26 +215,10 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi if len(inProgressTasks) < maxTasksInProgress { inProgressTasks = append(inProgressTasks, task) } else { - // put into the table wise pending tasks. - var tablePendingTasks []*ImportFileTask - var ok bool - tablePendingTasks, ok = tableWisePendingTasks.Get(tableName) - if !ok { - tablePendingTasks = []*ImportFileTask{} - } - tablePendingTasks = append(tablePendingTasks, task) - tableWisePendingTasks.Put(tableName, tablePendingTasks) + addToPendingTasks(task) } case FILE_IMPORT_NOT_STARTED: - // put into the table wise pending tasks. - var tablePendingTasks []*ImportFileTask - var ok bool - tablePendingTasks, ok = tableWisePendingTasks.Get(tableName) - if !ok { - tablePendingTasks = []*ImportFileTask{} - } - tablePendingTasks = append(tablePendingTasks, task) - tableWisePendingTasks.Put(tableName, tablePendingTasks) + addToPendingTasks(task) default: return nil, fmt.Errorf("unexpected status for task: %v: %v", task, taskStatus) } From 3022730b7c62eb444037400a095c0846625b1677 Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 18 Feb 2025 19:03:08 +0530 Subject: [PATCH 85/87] review comments --- .../cmd/importDataFileTaskPicker_test.go | 83 ++++++++++++++++++- 1 file changed, 79 insertions(+), 4 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker_test.go b/yb-voyager/cmd/importDataFileTaskPicker_test.go index aa5d1c6b45..c24a17ffc0 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker_test.go +++ b/yb-voyager/cmd/importDataFileTaskPicker_test.go @@ -144,6 +144,81 @@ func TestSequentialTaskPickerMarkTaskDone(t *testing.T) { assert.Error(t, err) } +func TestSequentialTaskPickerResumePicksInProgressTask(t *testing.T) { + ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) + testutils.FatalIfError(t, err) + + if ldataDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", ldataDir)) + } + if lexportDir != "" { + defer os.RemoveAll(fmt.Sprintf("%s/", lexportDir)) + } + + _, task1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.table1", 1) + testutils.FatalIfError(t, err) + _, task2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.table2", 2) + testutils.FatalIfError(t, err) + + tasks := []*ImportFileTask{ + task1, + task2, + } + + picker, err := NewSequentialTaskPicker(tasks, state) + testutils.FatalIfError(t, err) + + assert.True(t, picker.HasMoreTasks()) + + // no matter how many times we call NextTask, it should return the same task (first task) + for i := 0; i < 10; i++ { + task, err := picker.Pick() + assert.NoError(t, err) + assert.Equal(t, task1, task) + } + + // update the state of the first task to in progress + fbp, err := NewFileBatchProducer(task1, state) + testutils.FatalIfError(t, err) + batch, err := fbp.NextBatch() + assert.NoError(t, err) + err = batch.MarkInProgress() + assert.NoError(t, err) + taskState, err := state.GetFileImportState(task1.FilePath, task1.TableNameTup) + assert.NoError(t, err) + assert.Equal(t, FILE_IMPORT_IN_PROGRESS, taskState) + + // simulate restart by creating a new picker + slices.Reverse(tasks) // reorder the tasks so that the in progress task is at the end + picker, err = NewSequentialTaskPicker(tasks, state) + + // no matter how many times we call NextTask, it should return the same task (first task) + for i := 0; i < 10; i++ { + task, err := picker.Pick() + assert.NoError(t, err) + assert.Equal(t, task1, task) + } + + // mark the first task as done, now, the picker should return task2 + err = picker.MarkTaskAsDone(task1) + assert.NoError(t, err) + + for i := 0; i < 10; i++ { + task, err := picker.Pick() + assert.NoError(t, err) + assert.Equal(t, task2, task) + } + + // mark the second task as done, then the picker should not have any tasks anymore + err = picker.MarkTaskAsDone(task2) + assert.NoError(t, err) + assert.False(t, picker.HasMoreTasks()) + + // marking any task as done now should return an error + err = picker.MarkTaskAsDone(task1) + assert.Error(t, err) +} + func TestColocatedAwareRandomTaskPickerAdheresToMaxTasksInProgress(t *testing.T) { ldataDir, lexportDir, state, err := setupExportDirAndImportDependencies(2, 1024) testutils.FatalIfError(t, err) @@ -253,13 +328,13 @@ func TestColocatedAwareRandomTaskPickerMultipleTasksPerTableAdheresToMaxTasksInP } // multiple tasks for same table. - _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded", 1) + _, shardedTask1, err := createFileAndTask(lexportDir, "file1", ldataDir, "public.sharded1", 1) testutils.FatalIfError(t, err) - _, shardedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.sharded", 2) + _, shardedTask2, err := createFileAndTask(lexportDir, "file2", ldataDir, "public.sharded2", 2) testutils.FatalIfError(t, err) - _, colocatedTask1, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated", 3) + _, colocatedTask1, err := createFileAndTask(lexportDir, "file3", ldataDir, "public.colocated1", 3) testutils.FatalIfError(t, err) - _, colocatedTask2, err := createFileAndTask(lexportDir, "file4", ldataDir, "public.colocated", 3) + _, colocatedTask2, err := createFileAndTask(lexportDir, "file4", ldataDir, "public.colocated2", 3) testutils.FatalIfError(t, err) tasks := []*ImportFileTask{ From 7a09185ac489ea6c610ec378644ccb40fb3e6ffd Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 18 Feb 2025 19:04:45 +0530 Subject: [PATCH 86/87] comment --- yb-voyager/cmd/importDataFileTaskPicker.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index 97fc860fae..f7db89332a 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -290,6 +290,8 @@ func (c *ColocatedAwareRandomTaskPicker) PickTaskFromPendingTasks() (*ImportFile c.tableWisePendingTasks.Delete(tablePick) // reinitialize chooser because we have removed a table from the pending list, so weights will change. + // we need to update the weights every time the list of pending tasks change. + // We can't simply remove a choice, because the weights need to be rebalanced based on what is pending. if len(c.tableWisePendingTasks.Keys()) > 0 { err := c.initializeChooser() if err != nil { From 32ba5b9874bc9281f42857cda8b48f68b63a96be Mon Sep 17 00:00:00 2001 From: Aneesh Makala Date: Tue, 18 Feb 2025 22:05:47 +0530 Subject: [PATCH 87/87] initialize chooser only if pending tasks are there --- yb-voyager/cmd/importDataFileTaskPicker.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/yb-voyager/cmd/importDataFileTaskPicker.go b/yb-voyager/cmd/importDataFileTaskPicker.go index f7db89332a..cac3b524eb 100644 --- a/yb-voyager/cmd/importDataFileTaskPicker.go +++ b/yb-voyager/cmd/importDataFileTaskPicker.go @@ -231,9 +231,11 @@ func NewColocatedAwareRandomTaskPicker(maxTasksInProgress int, tasks []*ImportFi tableWisePendingTasks: tableWisePendingTasks, tableTypes: tableTypes, } - err = picker.initializeChooser() - if err != nil { - return nil, fmt.Errorf("initializing chooser: %w", err) + if len(picker.tableWisePendingTasks.Keys()) > 0 { + err = picker.initializeChooser() + if err != nil { + return nil, fmt.Errorf("initializing chooser: %w", err) + } } log.Infof("ColocatedAwareRandomTaskPicker initialized with params:%v", spew.Sdump(picker))