Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
45269: importccl: Parallelize avro import r=miretskiy a=miretskiy

Parallelize avro importer to improve its throughput (2.8x improvement).

Touches #40374.
Fixes #45097.

Release notes (performance): Faster avro import

45482: storage: integrate Concurrency Manager into Replica request path r=nvanbenschoten a=nvanbenschoten

Related to #41720.
Related to #44976.

This commit integrates the new concurrency package into the storage package. Each Replica is given a concurrency manager, which replaces its existing latch manager and txn wait queue. The change also uses the concurrency manager to simplify the role of the intent resolver. The intent resolver no longer directly handles WriteIntentErrors. As a result, we are able to delete the contention queue entirely.

With this change, all requests are now sequenced through the concurrency manager. When sequencing, latches are acquired and conflicting locks are detected. If any locks are found, the requests wait in lock wait-queues for the locks to be resolved. This is a major deviation from how things currently work because today, even with the contention queue, requests end up waiting for conflicting transactions to commit/abort in the txnWaitQueue after at least one RPC. Now, requests wait directly next to the intents/locks that they are waiting on and react directly to the resolution of these intents/locks.

Once requests are sequenced by the concurrency manager, they are theoretically fully isolated from all conflicting requests. However, this is not strictly true today because we have not yet pulled all replicated locks into the concurrency manager's lock table. We will do so in a future change. Until then, the concurrency manager maintains a notion of "intent discovery", which is integrated into the Replica-level concurrency retry loop.

Performance numbers will be published shortly. This will be followed by performance numbers using the SELECT FOR UPDATE locking (#40205) improvements that this change enables.

45484: sql: simplify connection state machine - stop tracking retry intent r=andreimatei a=andreimatei

Before this patch, the SQL connection state machine had an optimization:
if a transaction that hadn't used "SAVEPOINT cockroach_restart"
encountered a retriable error that we can't auto-retry, then we'd
release the txn's locks eagerly and enter the Aborted state. As opposed
to transactions that had used the "SAVEPOINT cockroach_restart", which
go to RestartWait.
This optimization is a significant complication for the state machine,
so this patch is removing it. All transactions now go to RestartWait,
and wait for a ROLLBACK to release the locks.

On the flip side, doing "RELEASE SAVEPOINT cockroach_restart" and
"ROLLBACK SAVEPOINT cockroach_restart" now works even for transactions
that haven't explicitly declared that savepoint, which is nice. Although
I don't promise I'll keep it working.

Release note: None

Co-authored-by: Yevgeniy Miretskiy <yevgeniy@cockroachlabs.com>
Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
  • Loading branch information
4 people committed Mar 2, 2020
4 parents ec05f0a + 7bd8e7d + 1899ad4 + 198e4e7 commit cf8cd92
Show file tree
Hide file tree
Showing 90 changed files with 2,145 additions and 2,700 deletions.
7 changes: 5 additions & 2 deletions pkg/ccl/importccl/import_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,9 +150,12 @@ func makeInputConverter(
case roachpb.IOFileFormat_PgDump:
return newPgDumpReader(ctx, kvCh, spec.Format.PgDump, spec.Tables, evalCtx)
case roachpb.IOFileFormat_Avro:
return newAvroInputReader(ctx, kvCh, singleTable, spec.Format.Avro, evalCtx)
return newAvroInputReader(
kvCh, singleTable, spec.Format.Avro, spec.WalltimeNanos,
int(spec.ReaderParallelism), evalCtx)
default:
return nil, errors.Errorf("Requested IMPORT format (%d) not supported by this node", spec.Format.Format)
return nil, errors.Errorf(
"Requested IMPORT format (%d) not supported by this node", spec.Format.Format)
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/importccl/import_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -577,7 +577,7 @@ func TestCSVImportCanBeResumed(t *testing.T) {
defer leaktest.AfterTest(t)()
defer setImportReaderParallelism(1)()
const batchSize = 5
defer TestingSetCsvInputReaderBatchSize(batchSize)()
defer TestingSetParallelImporterReaderBatchSize(batchSize)()
defer row.TestingSetDatumRowConverterBatchSize(2 * batchSize)()
jobs.DefaultAdoptInterval = 100 * time.Millisecond

Expand Down Expand Up @@ -683,7 +683,7 @@ func TestCSVImportCanBeResumed(t *testing.T) {
func TestCSVImportMarksFilesFullyProcessed(t *testing.T) {
defer leaktest.AfterTest(t)()
const batchSize = 5
defer TestingSetCsvInputReaderBatchSize(batchSize)()
defer TestingSetParallelImporterReaderBatchSize(batchSize)()
defer row.TestingSetDatumRowConverterBatchSize(2 * batchSize)()
jobs.DefaultAdoptInterval = 100 * time.Millisecond

Expand Down
104 changes: 58 additions & 46 deletions pkg/ccl/importccl/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"net/http/httptest"
"net/url"
"path/filepath"
"runtime"
"strings"
"sync"
"testing"
Expand Down Expand Up @@ -53,7 +52,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/pkg/errors"
"github.com/stretchr/testify/require"
"golang.org/x/sync/errgroup"
)

func TestImportData(t *testing.T) {
Expand Down Expand Up @@ -2123,9 +2121,8 @@ func TestImportIntoCSV(t *testing.T) {
sqlDB.Exec(t, "INSERT INTO t (a, b) VALUES ($1, $2)", i, v)
}

stripFilenameQuotes := strings.ReplaceAll(testFiles.files[0][1:len(testFiles.files[0])-1], "?", "\\?")
sqlDB.ExpectErr(
t, fmt.Sprintf("%s: row 1: expected 3 fields, got 2", stripFilenameQuotes),
t, "row 1: expected 3 fields, got 2",
fmt.Sprintf(`IMPORT INTO t (a, b, c) CSV DATA (%s)`, testFiles.files[0]),
)
})
Expand All @@ -2136,9 +2133,8 @@ func TestImportIntoCSV(t *testing.T) {
sqlDB.Exec(t, `CREATE TABLE t (a INT)`)
defer sqlDB.Exec(t, `DROP TABLE t`)

stripFilenameQuotes := strings.ReplaceAll(testFiles.files[0][1:len(testFiles.files[0])-1], "?", "\\?")
sqlDB.ExpectErr(
t, fmt.Sprintf("%s: row 1: expected 1 fields, got 2", stripFilenameQuotes),
t, "row 1: expected 1 fields, got 2",
fmt.Sprintf(`IMPORT INTO t (a) CSV DATA (%s)`, testFiles.files[0]),
)
})
Expand Down Expand Up @@ -2374,6 +2370,46 @@ func BenchmarkImport(b *testing.B) {
))
}

// a importRowProducer implementation that returns 'n' rows.
type csvBenchmarkStream struct {
n int
pos int
data [][]string
}

func (s *csvBenchmarkStream) Progress() float32 {
return float32(s.pos) / float32(s.n)
}

func (s *csvBenchmarkStream) Scan() bool {
s.pos++
return s.pos <= s.n
}

func (s *csvBenchmarkStream) Err() error {
return nil
}

func (s *csvBenchmarkStream) Skip() error {
return nil
}

func (s *csvBenchmarkStream) Row() (interface{}, error) {
return s.data[s.pos%len(s.data)], nil
}

var _ importRowProducer = &csvBenchmarkStream{}

// BenchmarkConvertRecord-16 1000000 2107 ns/op 56.94 MB/s 3600 B/op 101 allocs/op
// BenchmarkConvertRecord-16 500000 2106 ns/op 56.97 MB/s 3606 B/op 101 allocs/op
// BenchmarkConvertRecord-16 500000 2100 ns/op 57.14 MB/s 3606 B/op 101 allocs/op
// BenchmarkConvertRecord-16 500000 2286 ns/op 52.49 MB/s 3606 B/op 101 allocs/op
// BenchmarkConvertRecord-16 500000 2378 ns/op 50.46 MB/s 3606 B/op 101 allocs/op
// BenchmarkConvertRecord-16 500000 2427 ns/op 49.43 MB/s 3606 B/op 101 allocs/op
// BenchmarkConvertRecord-16 500000 2399 ns/op 50.02 MB/s 3606 B/op 101 allocs/op
// BenchmarkConvertRecord-16 500000 2365 ns/op 50.73 MB/s 3606 B/op 101 allocs/op
// BenchmarkConvertRecord-16 500000 2376 ns/op 50.49 MB/s 3606 B/op 101 allocs/op
// BenchmarkConvertRecord-16 500000 2390 ns/op 50.20 MB/s 3606 B/op 101 allocs/op
func BenchmarkConvertRecord(b *testing.B) {
ctx := context.TODO()

Expand Down Expand Up @@ -2429,58 +2465,31 @@ func BenchmarkConvertRecord(b *testing.B) {
if err != nil {
b.Fatal(err)
}
recordCh := make(chan csvRecord)
kvCh := make(chan row.KVBatch)
group := errgroup.Group{}

kvCh := make(chan row.KVBatch)
// no-op drain kvs channel.
go func() {
for range kvCh {
}
}()

c := &csvInputReader{
descr := tableDesc.TableDesc()
importCtx := &parallelImportContext{
evalCtx: &evalCtx,
tableDesc: descr,
kvCh: kvCh,
recordCh: recordCh,
tableDesc: tableDesc.TableDesc(),
}
// start up workers.
numWorkers := runtime.NumCPU()
for i := 0; i < numWorkers; i++ {
workerID := i
group.Go(func() error {
return c.convertRecordWorker(ctx, workerID)
})
}
const batchSize = 500

minEmitted := make([]int64, numWorkers)
batch := csvRecord{
file: "some/path/to/some/file/of/csv/data.tbl",
rowOffset: 1,
r: make([][]string, 0, batchSize),
minEmitted: &minEmitted,
}

b.ResetTimer()
for i := 0; i < b.N; i++ {
if len(batch.r) > batchSize {
recordCh <- batch
batch.r = make([][]string, 0, batchSize)
batch.rowOffset = int64(i)
minEmitted = make([]int64, numWorkers)
}

batch.r = append(batch.r, tpchLineItemDataRows[i%len(tpchLineItemDataRows)])
}
recordCh <- batch
close(recordCh)

if err := group.Wait(); err != nil {
b.Fatal(err)
producer := &csvBenchmarkStream{
n: b.N,
pos: 0,
data: tpchLineItemDataRows,
}
consumer := &csvRowConsumer{importCtx: importCtx, opts: &roachpb.CSVOptions{}}
b.ResetTimer()
require.NoError(b, runParallelImport(ctx, importCtx, &importFileContext{}, producer, consumer))
close(kvCh)
b.ReportAllocs()
}

// TestImportControlJob tests that PAUSE JOB, RESUME JOB, and CANCEL JOB
Expand Down Expand Up @@ -3556,8 +3565,11 @@ func TestImportAvro(t *testing.T) {
},
}

for _, test := range tests {
for i, test := range tests {
t.Run(test.name, func(t *testing.T) {
// Play a bit with producer/consumer batch sizes.
defer TestingSetParallelImporterReaderBatchSize(13 * i)()

_, err := sqlDB.DB.ExecContext(context.Background(), `DROP TABLE IF EXISTS simple CASCADE`)
require.NoError(t, err)

Expand Down
Loading

0 comments on commit cf8cd92

Please sign in to comment.