forked from cockroachdb/cockroach
/
joinreader.go
179 lines (152 loc) · 4.71 KB
/
joinreader.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
// Copyright 2016 The Cockroach Authors.
//
// 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.
//
// Author: Radu Berinde (radu@cockroachlabs.com)
package distsql
import (
"sync"
"golang.org/x/net/context"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/pkg/errors"
)
// TODO(radu): we currently create one batch at a time and run the KV operations
// on this node. In the future we may want to build separate batches for the
// nodes that "own" the respective ranges, and send out flows on those nodes.
const joinReaderBatchSize = 100
type joinReader struct {
readerBase
ctx context.Context
input RowSource
output RowReceiver
}
var _ processor = &joinReader{}
func newJoinReader(
flowCtx *FlowCtx, spec *JoinReaderSpec, input RowSource, output RowReceiver,
) (*joinReader, error) {
if spec.IndexIdx != 0 {
// TODO(radu): for now we only support joining with the primary index
return nil, errors.Errorf("join with index not implemented")
}
jr := &joinReader{
input: input,
output: output,
}
err := jr.readerBase.init(flowCtx, &spec.Table, int(spec.IndexIdx), spec.Filter,
spec.OutputColumns, false)
if err != nil {
return nil, err
}
jr.ctx = log.WithLogTagInt(jr.flowCtx.Context, "JoinReader", int(jr.desc.ID))
return jr, nil
}
func (jr *joinReader) generateKey(
row sqlbase.EncDatumRow, alloc *sqlbase.DatumAlloc, primaryKeyPrefix []byte,
) (roachpb.Key, error) {
index := jr.index
if len(row) < len(index.ColumnIDs) {
return nil, errors.Errorf("joinReader input has %d columns, expected at least %d",
len(row), len(jr.desc.PrimaryIndex.ColumnIDs))
}
row = row[:len(index.ColumnIDs)]
// Verify the types.
// TODO(radu): not strictly needed, perhaps enable only for tests.
for i, cid := range index.ColumnIDs {
colType := jr.desc.Columns[jr.colIdxMap[cid]].Type.Kind
if row[i].Type != colType {
return nil, errors.Errorf("joinReader input column %d has invalid type %s, expected %s",
i, row[i].Type, colType)
}
}
return sqlbase.MakeKeyFromEncDatums(row, &jr.desc, index, primaryKeyPrefix, alloc)
}
// mainLoop runs the mainLoop and returns any error.
// It does not close the output.
func (jr *joinReader) mainLoop() error {
primaryKeyPrefix := sqlbase.MakeIndexKeyPrefix(&jr.desc, jr.index.ID)
var alloc sqlbase.DatumAlloc
spans := make(roachpb.Spans, 0, joinReaderBatchSize)
ctx, span := tracing.ChildSpan(jr.ctx, "join reader")
defer tracing.FinishSpan(span)
log.VEventf(ctx, 1, "starting (filter: %s)", &jr.filter)
if log.V(1) {
defer log.Infof(ctx, "exiting")
}
for {
// TODO(radu): figure out how to send smaller batches if the source has
// a soft limit (perhaps send the batch out if we don't get a result
// within a certain amount of time).
for spans = spans[:0]; len(spans) < joinReaderBatchSize; {
row, err := jr.input.NextRow()
if err != nil {
return err
}
if row == nil {
if len(spans) == 0 {
return nil
}
break
}
key, err := jr.generateKey(row, &alloc, primaryKeyPrefix)
if err != nil {
return err
}
spans = append(spans, roachpb.Span{
Key: key,
EndKey: key.PrefixEnd(),
})
}
err := jr.fetcher.StartScan(jr.flowCtx.txn, spans, false /* no batch limits */, 0)
if err != nil {
log.Errorf(ctx, "scan error: %s", err)
return err
}
// TODO(radu): we are consuming all results from a fetch before starting
// the next batch. We could start the next batch early while we are
// outputting rows.
for {
outRow, err := jr.nextRow()
if err != nil {
return err
}
if outRow == nil {
// Done.
break
}
if log.V(3) {
log.Infof(ctx, "pushing row %s\n", outRow)
}
// Push the row to the output RowReceiver; stop if they don't need more
// rows.
if !jr.output.PushRow(outRow) {
log.VEventf(ctx, 1, "no more rows required")
return nil
}
}
if len(spans) != joinReaderBatchSize {
// This was the last batch.
return nil
}
}
}
// Run is part of the processor interface.
func (jr *joinReader) Run(wg *sync.WaitGroup) {
if wg != nil {
defer wg.Done()
}
err := jr.mainLoop()
jr.output.Close(err)
}