Skip to content

Commit

Permalink
Merge #44464
Browse files Browse the repository at this point in the history
44464: sqlbase: add round-trippable export and parsing for arrays and collated strings r=knz a=rohany

Work for #44322.
Fixes #33429.
Fixes #44414.
Closes #44669.

This PR adds support for ARRAY's and collated strings to be dumped with the
`tree.FmtExport` flag, which dumps datums in a string format that can be
round-tripped with `sqlbase.ParseDatumStringAs`.

This PR additionally performs some code cleanup and renaming
around various parsing functions within the `tree` package.
In particular, this PR removes the function `tree.ParseStringAs` which
was confusingly used sometimes instead of `sqlbase.ParseDatumStringAs`
due to it's handling of raw bytes. Instead, we introduce a new function
`sqlbase.ParseDatumStringAsWithRawBytes` to be more explicit about when
to use one parsing function over the other.

Release note (bug fix): This PR fixes bugs around `cockroach dump` and
`IMPORT`/`EXPORT` where columns of arrays or collated strings would
not be able to be roundtripped from cockroach to the dump and back to cockroach.

Co-authored-by: Rohan Yadav <[email protected]>
  • Loading branch information
craig[bot] and rohany committed Feb 5, 2020
2 parents 6f601e2 + dbfd995 commit 2859ad3
Show file tree
Hide file tree
Showing 19 changed files with 356 additions and 274 deletions.
46 changes: 46 additions & 0 deletions pkg/ccl/importccl/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1553,6 +1553,52 @@ func TestImportCSVStmt(t *testing.T) {
})
}

func TestExportImportRoundTrip(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
baseDir, cleanup := testutils.TempDir(t)
defer cleanup()
tc := testcluster.StartTestCluster(
t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ExternalIODir: baseDir}})
defer tc.Stopper().Stop(ctx)
conn := tc.Conns[0]
sqlDB := sqlutils.MakeSQLRunner(conn)

tests := []struct {
stmts string
tbl string
expected string
}{
// Note that the directory names that are being imported from and exported into
// need to differ across runs, so we let the test runner format the stmts field
// with a unique directory name per run.
{
stmts: `EXPORT INTO CSV 'nodelocal:///%[1]s' FROM SELECT ARRAY['a', 'b', 'c'];
IMPORT TABLE t (x TEXT[]) CSV DATA ('nodelocal:///%[1]s/n1.0.csv')`,
tbl: "t",
expected: `SELECT ARRAY['a', 'b', 'c']`,
},
{
stmts: `EXPORT INTO CSV 'nodelocal:///%[1]s' FROM SELECT ARRAY[b'abc', b'\141\142\143', b'\x61\x62\x63'];
IMPORT TABLE t (x BYTES[]) CSV DATA ('nodelocal:///%[1]s/n1.0.csv')`,
tbl: "t",
expected: `SELECT ARRAY[b'abc', b'\141\142\143', b'\x61\x62\x63']`,
},
{
stmts: `EXPORT INTO CSV 'nodelocal:///%[1]s' FROM SELECT 'dog' COLLATE en;
IMPORT TABLE t (x STRING COLLATE en) CSV DATA ('nodelocal:///%[1]s/n1.0.csv')`,
tbl: "t",
expected: `SELECT 'dog' COLLATE en`,
},
}

for i, test := range tests {
sqlDB.Exec(t, fmt.Sprintf(`DROP TABLE IF EXISTS %s`, test.tbl))
sqlDB.Exec(t, fmt.Sprintf(test.stmts, fmt.Sprintf("run%d", i)))
sqlDB.CheckQueryResults(t, fmt.Sprintf(`SELECT * FROM %s`, test.tbl), sqlDB.QueryStr(t, test.expected))
}
}

// TODO(adityamaru): Tests still need to be added incrementally as
// relevant IMPORT INTO logic is added. Some of them include:
// -> FK and constraint violation
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_avro.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ func nativeToDatum(
case string:
// We allow strings to be specified for any column, as
// long as we can convert the string value to the target type.
return tree.ParseStringAs(targetT, v, evalCtx)
return sqlbase.ParseDatumStringAs(targetT, v, evalCtx)
case map[string]interface{}:
for _, aT := range avroT {
// The value passed in is an avro schema. Extract
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_csv.go
Original file line number Diff line number Diff line change
Expand Up @@ -257,7 +257,7 @@ func (c *csvInputReader) convertRecordWorker(ctx context.Context, workerID int)
conv.Datums[datumIdx] = tree.DNull
} else {
var err error
conv.Datums[datumIdx], err = tree.ParseDatumStringAs(conv.VisibleColTypes[i], field, conv.EvalCtx)
conv.Datums[datumIdx], err = sqlbase.ParseDatumStringAs(conv.VisibleColTypes[i], field, conv.EvalCtx)
if err != nil {
err = wrapRowErr(err, batch.file, rowNum, pgcode.Syntax,
"parse %q as %s", col.Name, col.Type.SQLString())
Expand Down
10 changes: 7 additions & 3 deletions pkg/ccl/importccl/read_import_mysql.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,11 +218,15 @@ func mysqlValueToDatum(
}
}
}
return tree.ParseStringAs(desired, s, evalContext)
// This uses ParseDatumStringAsWithRawBytes instead of ParseDatumStringAs since mysql emits
// raw byte strings that do not use the same escaping as our ParseBytes
// function expects, and the difference between ParseStringAs and
// ParseDatumStringAs is whether or not it attempts to parse bytes.
return sqlbase.ParseDatumStringAsWithRawBytes(desired, s, evalContext)
case mysql.IntVal:
return tree.ParseStringAs(desired, string(v.Val), evalContext)
return sqlbase.ParseDatumStringAs(desired, string(v.Val), evalContext)
case mysql.FloatVal:
return tree.ParseStringAs(desired, string(v.Val), evalContext)
return sqlbase.ParseDatumStringAs(desired, string(v.Val), evalContext)
case mysql.HexVal:
v, err := v.HexDecode()
return tree.NewDBytes(tree.DBytes(v)), err
Expand Down
6 changes: 5 additions & 1 deletion pkg/ccl/importccl/read_import_mysqlout.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,11 @@ func (d *mysqloutfileReader) readFile(
} else if (!d.opts.HasEscape && field == "NULL") || d.opts.NullEncoding != nil && field == *d.opts.NullEncoding {
row = append(row, tree.DNull)
} else {
datum, err := tree.ParseStringAs(d.conv.VisibleColTypes[len(row)], field, d.conv.EvalCtx)
// This uses ParseDatumStringAsWithRawBytes instead of ParseDatumStringAs since mysql emits
// raw byte strings that do not use the same escaping as our ParseBytes
// function expects, and the difference between ParseStringAs and
// ParseDatumStringAs is whether or not it attempts to parse bytes.
datum, err := sqlbase.ParseDatumStringAsWithRawBytes(d.conv.VisibleColTypes[len(row)], field, d.conv.EvalCtx)
if err != nil {
col := d.conv.VisibleCols[len(row)]
return wrapRowErr(err, inputName, count, pgcode.Syntax,
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_pgcopy.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,7 +298,7 @@ func (d *pgCopyReader) readFile(
if s == nil {
d.conv.Datums[i] = tree.DNull
} else {
d.conv.Datums[i], err = tree.ParseDatumStringAs(d.conv.VisibleColTypes[i], *s, d.conv.EvalCtx)
d.conv.Datums[i], err = sqlbase.ParseDatumStringAs(d.conv.VisibleColTypes[i], *s, d.conv.EvalCtx)
if err != nil {
col := d.conv.VisibleCols[i]
return wrapRowErr(err, inputName, count, pgcode.Syntax,
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_pgdump.go
Original file line number Diff line number Diff line change
Expand Up @@ -574,7 +574,7 @@ func (m *pgDumpReader) readFile(
if s == nil {
conv.Datums[i] = tree.DNull
} else {
conv.Datums[i], err = tree.ParseDatumStringAs(conv.VisibleColTypes[i], *s, conv.EvalCtx)
conv.Datums[i], err = sqlbase.ParseDatumStringAs(conv.VisibleColTypes[i], *s, conv.EvalCtx)
if err != nil {
col := conv.VisibleCols[i]
return wrapRowErr(err, inputName, count, pgcode.Syntax,
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_workload.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ func makeDatumFromColOffset(
default:
data := col.Bytes().Get(rowIdx)
str := *(*string)(unsafe.Pointer(&data))
return tree.ParseDatumStringAs(hint, str, evalCtx)
return sqlbase.ParseDatumStringAs(hint, str, evalCtx)
}
}
return nil, errors.Errorf(
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/copy.go
Original file line number Diff line number Diff line change
Expand Up @@ -387,7 +387,7 @@ func (c *copyMachine) addRow(ctx context.Context, line []byte) error {
return err
}
}
d, err := tree.ParseStringAs(c.resultColumns[i].Typ, s, c.parsingEvalCtx)
d, err := sqlbase.ParseDatumStringAsWithRawBytes(c.resultColumns[i].Typ, s, c.parsingEvalCtx)
if err != nil {
return err
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/testutils/testcat/test_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/treeprinter"
Expand Down Expand Up @@ -999,7 +1000,7 @@ func (ts *TableStat) Histogram() []cat.HistogramBucket {
histogram := make([]cat.HistogramBucket, len(ts.js.HistogramBuckets))
for i := range histogram {
bucket := &ts.js.HistogramBuckets[i]
datum, err := tree.ParseStringAs(colType, bucket.UpperBound, &evalCtx)
datum, err := sqlbase.ParseDatumStringAs(colType, bucket.UpperBound, &evalCtx)
if err != nil {
panic(err)
}
Expand Down
5 changes: 1 addition & 4 deletions pkg/sql/sem/tree/constant.go
Original file line number Diff line number Diff line change
Expand Up @@ -540,10 +540,7 @@ func (expr *StrVal) ResolveAsType(ctx *SemaContext, typ *types.T) (Datum, error)
return ParseDByte(expr.s)
}

datum, err := parseStringAs(typ, expr.s, ctx)
if datum == nil && err == nil {
return nil, errors.AssertionFailedf("could not resolve %T %v into a %T", expr, expr, typ)
}
datum, err := ParseAndRequireString(typ, expr.s, ctx)
return datum, err
}

Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -3379,6 +3379,14 @@ func (d *DArray) Format(ctx *FmtCtx) {
return
}

// If we want to export arrays, we need to ensure that
// the datums within the arrays are formatted with enclosing quotes etc.
if ctx.HasFlags(FmtExport) {
oldFlags := ctx.flags
ctx.flags = oldFlags & ^FmtExport | FmtParsable
defer func() { ctx.flags = oldFlags }()
}

ctx.WriteString("ARRAY[")
comma := ""
for _, v := range d.Array {
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/sem/tree/eval_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ func TestEval(t *testing.T) {

switch typedExpr.ResolvedType().Family() {
case types.TupleFamily:
// ParseDatumStringAs doesn't handle tuples, so we have to convert them ourselves.
// ParseAndRequireString doesn't handle tuples, so we have to convert them ourselves.
var datums tree.Datums
// Fetch the original expression's tuple values.
tuple := typedExpr.(*tree.Tuple)
Expand All @@ -145,7 +145,7 @@ func TestEval(t *testing.T) {
t.Fatal(err)
}
// Now parse the new string as the expected type.
datum, err := tree.ParseDatumStringAs(expr.ResolvedType(), s, evalCtx)
datum, err := tree.ParseAndRequireString(expr.ResolvedType(), s, evalCtx)
if err != nil {
t.Errorf("%s: %s", err, s)
return err.Error()
Expand All @@ -154,7 +154,7 @@ func TestEval(t *testing.T) {
}
return tree.NewDTuple(typedExpr.ResolvedType(), datums...).String()
}
datum, err := tree.ParseDatumStringAs(typedExpr.ResolvedType(), res.String, evalCtx)
datum, err := tree.ParseAndRequireString(typedExpr.ResolvedType(), res.String, evalCtx)
if err != nil {
t.Errorf("%s: %s", err, res.String)
return err.Error()
Expand Down
6 changes: 1 addition & 5 deletions pkg/sql/sem/tree/parse_array.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/errors"
)

var enclosingError = pgerror.Newf(pgcode.InvalidTextRepresentation, "array must be enclosed in { and }")
Expand Down Expand Up @@ -137,10 +136,7 @@ func (p *parseState) parseElement() error {
}
}

d, err := parseStringAs(p.t, next, p.ctx)
if d == nil && err == nil {
return errors.AssertionFailedf("unknown type %s (%T)", p.t, p.t)
}
d, err := ParseAndRequireString(p.t, next, p.ctx)
if err != nil {
return err
}
Expand Down
43 changes: 4 additions & 39 deletions pkg/sql/sem/tree/parse_string.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,44 +15,9 @@ import (
"github.com/cockroachdb/errors"
)

// ParseStringAs reads s as type t. If t is Bytes or String, s is returned
// unchanged. Otherwise s is parsed with the given type's Parse func.
func ParseStringAs(t *types.T, s string, evalCtx *EvalContext) (Datum, error) {
var d Datum
var err error
switch t.Family() {
case types.BytesFamily:
d = NewDBytes(DBytes(s))
case types.CollatedStringFamily:
d, err = NewDCollatedString(s, t.Locale(), &evalCtx.CollationEnv)
case types.ArrayFamily:
d, err = ParseDArrayFromString(evalCtx, s, t.ArrayContents())
if err != nil {
return nil, err
}
default:
d, err = parseStringAs(t, s, evalCtx)
if d == nil && err == nil {
return nil, errors.AssertionFailedf("unknown type %s (%T)", t, t)
}
}
return d, err
}

// ParseDatumStringAs parses s as type t. This function is guaranteed to
// round-trip when printing a Datum with FmtExport.
func ParseDatumStringAs(t *types.T, s string, evalCtx *EvalContext) (Datum, error) {
switch t.Family() {
case types.BytesFamily:
return ParseDByte(s)
default:
return ParseStringAs(t, s, evalCtx)
}
}

// parseStringAs parses s as type t for simple types. Arrays and collated
// strings are not handled. nil, nil is returned if t is not a supported type.
func parseStringAs(t *types.T, s string, ctx ParseTimeContext) (Datum, error) {
// ParseAndRequireString parses s as type t for simple types. Arrays and collated
// strings are not handled.
func ParseAndRequireString(t *types.T, s string, ctx ParseTimeContext) (Datum, error) {
switch t.Family() {
case types.ArrayFamily:
return ParseDArrayFromString(ctx, s, t.ArrayContents())
Expand Down Expand Up @@ -96,6 +61,6 @@ func parseStringAs(t *types.T, s string, ctx ParseTimeContext) (Datum, error) {
case types.UuidFamily:
return ParseDUuidFromString(s)
default:
return nil, nil
return nil, errors.AssertionFailedf("unknown type %s (%T)", t, t)
}
}
Loading

0 comments on commit 2859ad3

Please sign in to comment.