github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/flowinfra/stream_encoder.go (about) 1 // Copyright 2016 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 package flowinfra 12 13 import ( 14 "context" 15 16 "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" 17 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 18 "github.com/cockroachdb/cockroach/pkg/sql/types" 19 "github.com/cockroachdb/errors" 20 ) 21 22 // PreferredEncoding is the encoding used for EncDatums that don't already have 23 // an encoding available. 24 const PreferredEncoding = sqlbase.DatumEncoding_ASCENDING_KEY 25 26 // StreamEncoder converts EncDatum rows into a sequence of ProducerMessage. 27 // 28 // Sample usage: 29 // se := StreamEncoder{} 30 // 31 // for { 32 // for ... { 33 // err := se.AddRow(...) 34 // ... 35 // } 36 // msg := se.FormMessage(nil) 37 // // Send out message. 38 // ... 39 // } 40 type StreamEncoder struct { 41 // infos is fully initialized when the first row is received. 42 infos []execinfrapb.DatumInfo 43 infosInitialized bool 44 45 rowBuf []byte 46 numEmptyRows int 47 metadata []execinfrapb.RemoteProducerMetadata 48 49 // headerSent is set after the first message (which contains the header) has 50 // been sent. 51 headerSent bool 52 // typingSent is set after the first message that contains any rows has been 53 // sent. 54 typingSent bool 55 alloc sqlbase.DatumAlloc 56 57 // Preallocated structures to avoid allocations. 58 msg execinfrapb.ProducerMessage 59 msgHdr execinfrapb.ProducerHeader 60 } 61 62 // HasHeaderBeenSent returns whether the header has been sent. 63 func (se *StreamEncoder) HasHeaderBeenSent() bool { 64 return se.headerSent 65 } 66 67 // SetHeaderFields sets the header fields. 68 func (se *StreamEncoder) SetHeaderFields(flowID execinfrapb.FlowID, streamID execinfrapb.StreamID) { 69 se.msgHdr.FlowID = flowID 70 se.msgHdr.StreamID = streamID 71 } 72 73 // Init initializes the encoder. 74 func (se *StreamEncoder) Init(types []*types.T) { 75 se.infos = make([]execinfrapb.DatumInfo, len(types)) 76 for i := range types { 77 se.infos[i].Type = types[i] 78 } 79 } 80 81 // AddMetadata encodes a metadata message. Unlike AddRow(), it cannot fail. This 82 // is important for the caller because a failure to encode a piece of metadata 83 // (particularly one that contains an error) would not be recoverable. 84 // 85 // Metadata records lose their ordering wrt the data rows. The convention is 86 // that the StreamDecoder will return them first, before the data rows, thus 87 // ensuring that rows produced _after_ an error are not received _before_ the 88 // error. 89 func (se *StreamEncoder) AddMetadata(ctx context.Context, meta execinfrapb.ProducerMetadata) { 90 se.metadata = append(se.metadata, execinfrapb.LocalMetaToRemoteProducerMeta(ctx, meta)) 91 } 92 93 // AddRow encodes a message. 94 func (se *StreamEncoder) AddRow(row sqlbase.EncDatumRow) error { 95 if se.infos == nil { 96 panic("Init not called") 97 } 98 if len(se.infos) != len(row) { 99 return errors.Errorf("inconsistent row length: expected %d, got %d", len(se.infos), len(row)) 100 } 101 if !se.infosInitialized { 102 // First row. Initialize encodings. 103 for i := range row { 104 enc, ok := row[i].Encoding() 105 if !ok { 106 enc = PreferredEncoding 107 } 108 sType := se.infos[i].Type 109 if enc != sqlbase.DatumEncoding_VALUE && 110 (sqlbase.HasCompositeKeyEncoding(sType) || sqlbase.MustBeValueEncoded(sType)) { 111 // Force VALUE encoding for composite types (key encodings may lose data). 112 enc = sqlbase.DatumEncoding_VALUE 113 } 114 se.infos[i].Encoding = enc 115 } 116 se.infosInitialized = true 117 } 118 if len(row) == 0 { 119 se.numEmptyRows++ 120 return nil 121 } 122 for i := range row { 123 var err error 124 se.rowBuf, err = row[i].Encode(se.infos[i].Type, &se.alloc, se.infos[i].Encoding, se.rowBuf) 125 if err != nil { 126 return err 127 } 128 } 129 return nil 130 } 131 132 // FormMessage populates a message containing the rows added since the last call 133 // to FormMessage. The returned ProducerMessage should be treated as immutable. 134 func (se *StreamEncoder) FormMessage(ctx context.Context) *execinfrapb.ProducerMessage { 135 msg := &se.msg 136 msg.Header = nil 137 msg.Data.RawBytes = se.rowBuf 138 msg.Data.NumEmptyRows = int32(se.numEmptyRows) 139 msg.Data.Metadata = make([]execinfrapb.RemoteProducerMetadata, len(se.metadata)) 140 copy(msg.Data.Metadata, se.metadata) 141 se.metadata = se.metadata[:0] 142 143 if !se.headerSent { 144 msg.Header = &se.msgHdr 145 se.headerSent = true 146 } 147 if !se.typingSent { 148 if se.infosInitialized { 149 msg.Typing = se.infos 150 se.typingSent = true 151 } 152 } else { 153 msg.Typing = nil 154 } 155 156 se.rowBuf = se.rowBuf[:0] 157 se.numEmptyRows = 0 158 return msg 159 }