Skip to content

Commit

Permalink
distsql: support tuples
Browse files Browse the repository at this point in the history
Closes cockroachdb#15938.

Release note (performance improvement): using tuples in a query no
longer reverts you to single node local SQL execution.
  • Loading branch information
Arjun Narayan committed May 29, 2018
1 parent c0c6e90 commit 7e12e2c
Show file tree
Hide file tree
Showing 15 changed files with 416 additions and 208 deletions.
16 changes: 1 addition & 15 deletions pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/distsqlplan"
"github.com/cockroachdb/cockroach/pkg/sql/distsqlrun"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/types"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util"
Expand Down Expand Up @@ -280,15 +279,6 @@ func newQueryNotSupportedErrorf(format string, args ...interface{}) error {
var mutationsNotSupportedError = newQueryNotSupportedError("mutations not supported")
var setNotSupportedError = newQueryNotSupportedError("SET / SET CLUSTER SETTING should never distribute")

// leafType returns the element type if the given type is an array, and the type
// itself otherwise.
func leafType(t types.T) types.T {
if a, ok := t.(types.TArray); ok {
return leafType(a.Typ)
}
return t
}

// checkSupportForNode returns a distRecommendation (as described above) or an
// error if the plan subtree is not supported by DistSQL.
// TODO(radu): add tests for this.
Expand All @@ -301,11 +291,7 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio
return dsp.checkSupportForNode(n.source.plan)

case *renderNode:
for i, e := range n.render {
typ := n.columns[i].Typ
if leafType(typ).FamilyEqual(types.FamTuple) {
return 0, newQueryNotSupportedErrorf("unsupported render type %s", typ)
}
for _, e := range n.render {
if err := dsp.checkExpr(e); err != nil {
return 0, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/distsqlrun/hash_row_container.go
Original file line number Diff line number Diff line change
Expand Up @@ -417,7 +417,7 @@ func (h *hashDiskRowContainer) Init(
h.columnEncoder.init(types, storedEqCols)
// Provide the diskRowContainer with an ordering on the equality columns of
// the rows that we will store. This will result in rows with the
// same equality columns ocurring contiguously in the keyspace.
// same equality columns occurring contiguously in the keyspace.
ordering := make(sqlbase.ColumnOrdering, len(storedEqCols))
for i := range ordering {
ordering[i] = sqlbase.ColumnOrderInfo{
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/distsqlrun/processors.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ type ProcOutputHelper struct {
//
// If renderExprs is set, these types correspond to the types of those
// expressions.
// If outpuCols is set, these types correspond to the types of
// If outputCols is set, these types correspond to the types of
// those columns.
// If neither is set, this is the internal schema of the processor.
outputTypes []sqlbase.ColumnType
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/distsqlrun/version_history.txt
Original file line number Diff line number Diff line change
Expand Up @@ -62,3 +62,6 @@
- Enhancements to lookup joins. They now support joining against secondary
indexes as well as left outer joins. Left join support required two
additional fields on JoinReaderSpec: index_filter_expr and type.
- Version: 15 (MinAcceptedVersion: 6)
- Add support for processing queries with tuples in DistSQL. Old versions
would not recognize the new tuple field in the proto.
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/select
Original file line number Diff line number Diff line change
Expand Up @@ -249,7 +249,7 @@ SELECT * FROM xyzw ORDER BY x OFFSET 1 + 0.0
----
4 5 6 7

query T
query T rowsort
SELECT (x,y) FROM xyzw
----
(1,2)
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -2609,6 +2609,10 @@ func (*DTuple) AmbiguousFormat() bool { return false }
// Format implements the NodeFormatter interface.
// TODO(bram): We don't format tuples in the same way as postgres. See #25522.
func (d *DTuple) Format(ctx *FmtCtx) {
if ctx.HasFlags(FmtParsable) && (len(d.D) == 0) {
ctx.WriteString("ROW()")
return
}
ctx.FormatNode(&d.D)
}

Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/sem/tree/format_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,6 +194,8 @@ func TestFormatExpr(t *testing.T) {
`unique_rowid() + 123:::INT`},
{`sqrt(123.0) + 456`, tree.FmtParsable,
`sqrt(123.0:::DECIMAL) + 456:::DECIMAL`},
{`ROW()`, tree.FmtParsable, `ROW()`},
{`ROW()`, tree.FmtSimple, `()`},
{`now() + interval '3s'`, tree.FmtSimple,
`now() + '3s'`},
{`now() + interval '3s'`, tree.FmtParsable,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/sqlbase/encoded_datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,7 @@ func (ed *EncDatum) EnsureDecoded(typ *ColumnType, a *DatumAlloc) error {
panic(fmt.Sprintf("unknown encoding %s", ed.encoding))
}
if err != nil {
return err
return errors.Wrapf(err, "error decoding %d bytes", len(ed.encoded))
}
if len(rem) != 0 {
ed.Datum = nil
Expand Down
38 changes: 38 additions & 0 deletions pkg/sql/sqlbase/encoded_datum_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/types"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
Expand Down Expand Up @@ -481,3 +482,40 @@ func TestEncDatumRowAlloc(t *testing.T) {
}
}
}

func TestValueEncodeDecodeTuple(t *testing.T) {
rng, seed := randutil.NewPseudoRand()
tests := make([]tree.Datum, 1000)
colTypes := make([]ColumnType, 1000)

for i := range tests {
colTypes[i] = ColumnType{SemanticType: ColumnType_TUPLE}

len := rng.Intn(5)
colTypes[i].TupleContents = make([]ColumnType, len)
for j := range colTypes[i].TupleContents {
colTypes[i].TupleContents[j] = RandColumnType(rng)
}
tests[i] = RandDatum(rng, colTypes[i], true)
}

for i, test := range tests {

buf, err := encodeTuple(test.(*tree.DTuple), nil, encoding.NoColumnID, nil)
if err != nil {
t.Fatalf("seed %d: encoding tuple %v with types %v failed with error: %v",
seed, test, colTypes[i], err)
}
var decodedTuple tree.Datum
decodedTuple, buf, err = decodeTuple(&DatumAlloc{}, test.ResolvedType().(types.TTuple), buf)
if err != nil {
t.Fatalf("seed %d: decoding tuple %v with types %v failed with error: %v",
seed, test, colTypes[i], err)
}

if cmp := decodedTuple.Compare(&tree.EvalContext{}, test); cmp != 0 {
t.Fatalf("seed %d: encoded %+v, decoded %+v, expected equal, received comparison: %d", seed, test, decodedTuple, cmp)
}
}

}
26 changes: 25 additions & 1 deletion pkg/sql/sqlbase/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -652,7 +652,7 @@ func DatumTypeHasCompositeKeyEncoding(typ types.T) bool {
// MustBeValueEncoded returns true if columns of the given kind can only be value
// encoded.
func MustBeValueEncoded(semanticType ColumnType_SemanticType) bool {
return semanticType == ColumnType_ARRAY || semanticType == ColumnType_JSON
return semanticType == ColumnType_ARRAY || semanticType == ColumnType_JSON || semanticType == ColumnType_TUPLE
}

// HasOldStoredColumns returns whether the index has stored columns in the old
Expand Down Expand Up @@ -2305,6 +2305,9 @@ func DatumTypeToColumnSemanticType(ptyp types.T) (ColumnType_SemanticType, error
if ptyp.FamilyEqual(types.FamCollatedString) {
return ColumnType_COLLATEDSTRING, nil
}
if ptyp.FamilyEqual(types.FamTuple) {
return ColumnType_TUPLE, nil
}
if wrapper, ok := ptyp.(types.TOidWrapper); ok {
return DatumTypeToColumnSemanticType(wrapper.T)
}
Expand All @@ -2330,6 +2333,17 @@ func DatumTypeToColumnType(ptyp types.T) (ColumnType, error) {
cs := t.Typ.(types.TCollatedString)
ctyp.Locale = &cs.Locale
}
case types.TTuple:
ctyp.SemanticType = ColumnType_TUPLE
ctyp.TupleContents = make([]ColumnType, len(t.Types))
for i, tc := range t.Types {
var err error
ctyp.TupleContents[i], err = DatumTypeToColumnType(tc)
if err != nil {
return ColumnType{}, err
}
}
return ctyp, nil
default:
semanticType, err := DatumTypeToColumnSemanticType(ptyp)
if err != nil {
Expand Down Expand Up @@ -2372,6 +2386,8 @@ func columnSemanticTypeToDatumType(c *ColumnType, k ColumnType_SemanticType) typ
return types.INet
case ColumnType_JSON:
return types.JSON
case ColumnType_TUPLE:
return types.FamTuple
case ColumnType_COLLATEDSTRING:
if c.Locale == nil {
panic("locale is required for COLLATEDSTRING")
Expand All @@ -2397,6 +2413,14 @@ func (c *ColumnType) ToDatumType() types.T {
switch c.SemanticType {
case ColumnType_ARRAY:
return types.TArray{Typ: columnSemanticTypeToDatumType(c, *c.ArrayContents)}
case ColumnType_TUPLE:
datums := types.TTuple{
Types: make([]types.T, len(c.TupleContents)),
}
for i := range c.TupleContents {
datums.Types[i] = c.TupleContents[i].ToDatumType()
}
return datums
default:
return columnSemanticTypeToDatumType(c, c.SemanticType)
}
Expand Down
Loading

0 comments on commit 7e12e2c

Please sign in to comment.