Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
31292: *: move RowFetcher and co to a new package, row r=jordanlewis a=jordanlewis

This was purely mechanical renaming. The purpose is to lessen the width
of the sqlbase package, which is imported in many places.

The new package now depends on sqlbase, but far fewer packages consume
it than sqlbase.

Major changes:

RowFetcher -> row.Fetcher
RowUpdater -> row.Updater
RowDeleter -> row.Deleter

FK and cascade stuff -> row package

The only unfortunate thing here is that the FK enums all begin with
`row` now (e.g. `row.CheckDeletes`). It would be better if those got a
different package name, like `fk`, but I'll leave that refactor for
another time.

Somewhat related to cockroachdb#30001.

cc @dt, @benesch 

Release note: None

Co-authored-by: Jordan Lewis <[email protected]>
  • Loading branch information
craig[bot] and jordanlewis committed Oct 12, 2018
2 parents 44db6b0 + bcc52bc commit 755d933
Show file tree
Hide file tree
Showing 50 changed files with 1,135 additions and 976 deletions.
3 changes: 2 additions & 1 deletion pkg/ccl/changefeedccl/changefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/bufalloc"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -71,7 +72,7 @@ func kvsToRows(
) func(context.Context) ([]emitEntry, error) {
rfCache := newRowFetcherCache(leaseMgr)

var kvs sqlbase.SpanKVFetcher
var kvs row.SpanKVFetcher
appendEmitEntryForKV := func(
ctx context.Context, output []emitEntry, kv roachpb.KeyValue, schemaTimestamp hlc.Timestamp,
) ([]emitEntry, error) {
Expand Down
13 changes: 7 additions & 6 deletions pkg/ccl/changefeedccl/rowfetcher_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand All @@ -21,20 +22,20 @@ import (

// rowFetcherCache maintains a cache of single table RowFetchers. Given a key
// with an mvcc timestamp, it retrieves the correct TableDescriptor for that key
// and returns a RowFetcher initialized with that table. This RowFetcher's
// and returns a Fetcher initialized with that table. This Fetcher's
// StartScanFrom can be used to turn that key (or all the keys making up the
// column families of one row) into a row.
type rowFetcherCache struct {
leaseMgr *sql.LeaseManager
fetchers map[*sqlbase.TableDescriptor]*sqlbase.RowFetcher
fetchers map[*sqlbase.TableDescriptor]*row.Fetcher

a sqlbase.DatumAlloc
}

func newRowFetcherCache(leaseMgr *sql.LeaseManager) *rowFetcherCache {
return &rowFetcherCache{
leaseMgr: leaseMgr,
fetchers: make(map[*sqlbase.TableDescriptor]*sqlbase.RowFetcher),
fetchers: make(map[*sqlbase.TableDescriptor]*row.Fetcher),
}
}

Expand Down Expand Up @@ -80,7 +81,7 @@ func (c *rowFetcherCache) TableDescForKey(

func (c *rowFetcherCache) RowFetcherForTableDesc(
tableDesc *sqlbase.TableDescriptor,
) (*sqlbase.RowFetcher, error) {
) (*row.Fetcher, error) {
if rf, ok := c.fetchers[tableDesc]; ok {
return rf, nil
}
Expand All @@ -93,10 +94,10 @@ func (c *rowFetcherCache) RowFetcherForTableDesc(
valNeededForCol.Add(colIdx)
}

var rf sqlbase.RowFetcher
var rf row.Fetcher
if err := rf.Init(
false /* reverse */, false /* returnRangeInfo */, false /* isCheck */, &c.a,
sqlbase.RowFetcherTableArgs{
row.FetcherTableArgs{
Spans: tableDesc.AllIndexSpans(),
Desc: tableDesc,
Index: &tableDesc.PrimaryIndex,
Expand Down
7 changes: 3 additions & 4 deletions pkg/ccl/importccl/import_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
Expand Down Expand Up @@ -363,10 +364,8 @@ func (r fkResolver) LookupSchema(
}

// Implements the sql.SchemaResolver interface.
func (r fkResolver) LookupTableByID(
ctx context.Context, id sqlbase.ID,
) (sqlbase.TableLookup, error) {
return sqlbase.TableLookup{}, errSchemaResolver
func (r fkResolver) LookupTableByID(ctx context.Context, id sqlbase.ID) (row.TableLookup, error) {
return row.TableLookup{}, errSchemaResolver
}

const csvDatabaseName = "csv"
Expand Down
23 changes: 12 additions & 11 deletions pkg/ccl/importccl/load.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sem/transform"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
Expand Down Expand Up @@ -97,7 +98,7 @@ func Load(

var currentCmd bytes.Buffer
scanner := bufio.NewReader(r)
var ri sqlbase.RowInserter
var ri row.Inserter
var defaultExprs []tree.TypedExpr
var cols []sqlbase.ColumnDescriptor
var tableDesc *sqlbase.TableDescriptor
Expand Down Expand Up @@ -181,7 +182,7 @@ func Load(
}
}

ri, err = sqlbase.MakeRowInserter(nil, tableDesc, nil, tableDesc.Columns,
ri, err = row.MakeInserter(nil, tableDesc, nil, tableDesc.Columns,
true, &sqlbase.DatumAlloc{})
if err != nil {
return backupccl.BackupDescriptor{}, errors.Wrap(err, "make row inserter")
Expand Down Expand Up @@ -256,7 +257,7 @@ func insertStmtToKVs(
defaultExprs []tree.TypedExpr,
cols []sqlbase.ColumnDescriptor,
evalCtx tree.EvalContext,
ri sqlbase.RowInserter,
ri row.Inserter,
stmt *tree.Insert,
f func(roachpb.KeyValue),
) error {
Expand Down Expand Up @@ -293,18 +294,18 @@ func insertStmtToKVs(
Cols: tableDesc.Columns,
}
for _, tuple := range values.Rows {
row := make([]tree.Datum, len(tuple))
insertRow := make([]tree.Datum, len(tuple))
for i, expr := range tuple {
if expr == tree.DNull {
row[i] = tree.DNull
insertRow[i] = tree.DNull
continue
}
c, ok := expr.(tree.Constant)
if !ok {
return errors.Errorf("unsupported expr: %q", expr)
}
var err error
row[i], err = c.ResolveAsType(nil, tableDesc.Columns[i].Type.ToDatumType())
insertRow[i], err = c.ResolveAsType(nil, tableDesc.Columns[i].Type.ToDatumType())
if err != nil {
return err
}
Expand All @@ -314,16 +315,16 @@ func insertStmtToKVs(
var computeExprs []tree.TypedExpr
var computedCols []sqlbase.ColumnDescriptor

row, err := sql.GenerateInsertRow(
defaultExprs, computeExprs, cols, computedCols, evalCtx, tableDesc, row, &computedIVarContainer,
insertRow, err := sql.GenerateInsertRow(
defaultExprs, computeExprs, cols, computedCols, evalCtx, tableDesc, insertRow, &computedIVarContainer,
)
if err != nil {
return errors.Wrapf(err, "process insert %q", row)
return errors.Wrapf(err, "process insert %q", insertRow)
}
// TODO(bram): Is the checking of FKs here required? If not, turning them
// off may provide a speed boost.
if err := ri.InsertRow(ctx, b, row, true, sqlbase.CheckFKs, false /* traceKV */); err != nil {
return errors.Wrapf(err, "insert %q", row)
if err := ri.InsertRow(ctx, b, insertRow, true, row.CheckFKs, false /* traceKV */); err != nil {
return errors.Wrapf(err, "insert %q", insertRow)
}
}
return nil
Expand Down
11 changes: 6 additions & 5 deletions pkg/ccl/importccl/read_import_proc.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/distsqlrun"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
Expand Down Expand Up @@ -200,7 +201,7 @@ type rowConverter struct {

// The rest of these are derived from tableDesc, just cached here.
hidden int
ri sqlbase.RowInserter
ri row.Inserter
evalCtx *tree.EvalContext
cols []sqlbase.ColumnDescriptor
visibleCols []sqlbase.ColumnDescriptor
Expand All @@ -220,7 +221,7 @@ func newRowConverter(
evalCtx: evalCtx,
}

ri, err := sqlbase.MakeRowInserter(nil /* txn */, tableDesc, nil, /* fkTables */
ri, err := row.MakeInserter(nil /* txn */, tableDesc, nil, /* fkTables */
tableDesc.Columns, false /* checkFKs */, &sqlbase.DatumAlloc{})
if err != nil {
return nil, errors.Wrap(err, "make row inserter")
Expand Down Expand Up @@ -293,7 +294,7 @@ func (c *rowConverter) row(ctx context.Context, fileIndex int32, rowIndex int64)
var computeExprs []tree.TypedExpr
var computedCols []sqlbase.ColumnDescriptor

row, err := sql.GenerateInsertRow(
insertRow, err := sql.GenerateInsertRow(
c.defaultExprs, computeExprs, c.cols, computedCols, *c.evalCtx, c.tableDesc, c.datums, &c.computedIVarContainer)
if err != nil {
return errors.Wrapf(err, "generate insert row")
Expand All @@ -304,9 +305,9 @@ func (c *rowConverter) row(ctx context.Context, fileIndex int32, rowIndex int64)
kv.Value.InitChecksum(kv.Key)
c.kvBatch = append(c.kvBatch, kv)
}),
row,
insertRow,
true, /* ignoreConflicts */
sqlbase.SkipFKs,
row.SkipFKs,
false, /* traceKV */
); err != nil {
return errors.Wrapf(err, "insert row")
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/settingsworker.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand All @@ -34,7 +35,7 @@ func (s *Server) refreshSettings() {

a := &sqlbase.DatumAlloc{}
settingsTablePrefix := keys.MakeTablePrefix(uint32(tbl.ID))
colIdxMap := sqlbase.ColIDtoRowIndexFromCols(tbl.Columns)
colIdxMap := row.ColIDtoRowIndexFromCols(tbl.Columns)

processKV := func(ctx context.Context, kv roachpb.KeyValue, u settings.Updater) error {
if !bytes.HasPrefix(kv.Key, settingsTablePrefix) {
Expand Down
29 changes: 15 additions & 14 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/backfill"
"github.com/cockroachdb/cockroach/pkg/sql/distsqlrun"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -216,7 +217,7 @@ func (sc *SchemaChanger) truncateIndexes(
alloc := &sqlbase.DatumAlloc{}
for _, desc := range dropped {
var resume roachpb.Span
for row, done := int64(0), false; !done; row += chunkSize {
for rowIdx, done := int64(0), false; !done; rowIdx += chunkSize {
// First extend the schema change lease.
if err := sc.ExtendLease(ctx, lease); err != nil {
return err
Expand All @@ -225,7 +226,7 @@ func (sc *SchemaChanger) truncateIndexes(
resumeAt := resume
if log.V(2) {
log.Infof(ctx, "drop index (%d, %d) at row: %d, span: %s",
sc.tableID, sc.mutationID, row, resume)
sc.tableID, sc.mutationID, rowIdx, resume)
}
if err := sc.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {
if fn := sc.execCfg.DistSQLRunTestingKnobs.RunBeforeBackfillChunk; fn != nil {
Expand All @@ -244,8 +245,8 @@ func (sc *SchemaChanger) truncateIndexes(
return err
}

rd, err := sqlbase.MakeRowDeleter(
txn, tableDesc, nil, nil, sqlbase.SkipFKs, nil /* *tree.EvalContext */, alloc,
rd, err := row.MakeDeleter(
txn, tableDesc, nil, nil, row.SkipFKs, nil /* *tree.EvalContext */, alloc,
)
if err != nil {
return err
Expand Down Expand Up @@ -400,12 +401,12 @@ func (sc *SchemaChanger) distBackfill(
// backfiller processor.
var otherTableDescs []sqlbase.TableDescriptor
if backfillType == columnBackfill {
fkTables, err := sqlbase.TablesNeededForFKs(
fkTables, err := row.TablesNeededForFKs(
ctx,
*tableDesc,
sqlbase.CheckUpdates,
sqlbase.NoLookup,
sqlbase.NoCheckPrivilege,
row.CheckUpdates,
row.NoLookup,
row.NoCheckPrivilege,
nil, /* AnalyzeExprFunction */
)
if err != nil {
Expand Down Expand Up @@ -592,12 +593,12 @@ func columnBackfillInTxn(
// otherTableDescs contains any other table descriptors required by the
// backfiller processor.
var otherTableDescs []sqlbase.TableDescriptor
fkTables, err := sqlbase.TablesNeededForFKs(
fkTables, err := row.TablesNeededForFKs(
ctx,
*tableDesc,
sqlbase.CheckUpdates,
sqlbase.NoLookup,
sqlbase.NoCheckPrivilege,
row.CheckUpdates,
row.NoLookup,
row.NoCheckPrivilege,
nil, /* AnalyzeExprFunction */
)
if err != nil {
Expand Down Expand Up @@ -656,8 +657,8 @@ func indexTruncateInTxn(
idx := tableDesc.Mutations[0].GetIndex()
var sp roachpb.Span
for done := false; !done; done = sp.Key == nil {
rd, err := sqlbase.MakeRowDeleter(
txn, tableDesc, nil, nil, sqlbase.SkipFKs, nil /* *tree.EvalContext */, alloc,
rd, err := row.MakeDeleter(
txn, tableDesc, nil, nil, row.SkipFKs, nil /* *tree.EvalContext */, alloc,
)
if err != nil {
return err
Expand Down
Loading

0 comments on commit 755d933

Please sign in to comment.