@@ -8,13 +8,13 @@ package changefeedccl
8
8
import (
9
9
"context"
10
10
"fmt"
11
- "strings "
11
+ "time "
12
12
13
13
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent"
14
14
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
15
15
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedpb"
16
- "github.com/cockroachdb/cockroach/pkg/geo"
17
16
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
17
+ "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
18
18
"github.com/cockroachdb/cockroach/pkg/util/hlc"
19
19
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
20
20
"github.com/cockroachdb/errors"
@@ -227,7 +227,7 @@ func encodeRowToRecord(row cdcevent.Row) (*changefeedpb.Record, error) {
227
227
}
228
228
record := & changefeedpb.Record {Values : make (map [string ]* changefeedpb.Value , row .NumValueColumns ())}
229
229
if err := row .ForEachColumn ().Datum (func (d tree.Datum , col cdcevent.ResultColumn ) error {
230
- val , err := datumToProtoValue (d )
230
+ val , err := datumToProtoValue (d , sessiondatapb. DataConversionConfig {}, time . UTC )
231
231
if err != nil {
232
232
return err
233
233
}
@@ -244,7 +244,7 @@ func buildKeyMessage(row cdcevent.Row) (*changefeedpb.Key, error) {
244
244
keyMap := make (map [string ]* changefeedpb.Value , row .NumKeyColumns ())
245
245
246
246
if err := row .ForEachKeyColumn ().Datum (func (d tree.Datum , col cdcevent.ResultColumn ) error {
247
- val , err := datumToProtoValue (d )
247
+ val , err := datumToProtoValue (d , sessiondatapb. DataConversionConfig {}, time . UTC )
248
248
if err != nil {
249
249
return err
250
250
}
@@ -259,12 +259,16 @@ func buildKeyMessage(row cdcevent.Row) (*changefeedpb.Key, error) {
259
259
// datumToProtoValue converts a tree.Datum into a changefeedpb.Value.
260
260
// It handles all common CockroachDB datum types and maps them to their
261
261
// corresponding protobuf representation.
262
- func datumToProtoValue (d tree.Datum ) (* changefeedpb.Value , error ) {
262
+ func datumToProtoValue (
263
+ d tree.Datum , dcc sessiondatapb.DataConversionConfig , loc * time.Location ,
264
+ ) (* changefeedpb.Value , error ) {
263
265
d = tree .UnwrapDOidWrapper (d )
266
+
264
267
if d == tree .DNull {
265
268
return nil , nil
266
269
}
267
270
switch v := d .(type ) {
271
+
268
272
case * tree.DBool :
269
273
return & changefeedpb.Value {Value : & changefeedpb.Value_BoolValue {BoolValue : bool (* v )}}, nil
270
274
case * tree.DInt :
@@ -285,7 +289,7 @@ func datumToProtoValue(d tree.Datum) (*changefeedpb.Value, error) {
285
289
case * tree.DArray :
286
290
elems := make ([]* changefeedpb.Value , 0 , v .Len ())
287
291
for _ , elt := range v .Array {
288
- pv , err := datumToProtoValue (elt )
292
+ pv , err := datumToProtoValue (elt , sessiondatapb. DataConversionConfig {}, time . UTC )
289
293
if err != nil {
290
294
return nil , err
291
295
}
@@ -297,7 +301,7 @@ func datumToProtoValue(d tree.Datum) (*changefeedpb.Value, error) {
297
301
labels := v .ResolvedType ().TupleLabels ()
298
302
records := make (map [string ]* changefeedpb.Value , len (v .D ))
299
303
for i , elem := range v .D {
300
- pv , err := datumToProtoValue (elem )
304
+ pv , err := datumToProtoValue (elem , sessiondatapb. DataConversionConfig {}, time . UTC )
301
305
if err != nil {
302
306
return nil , err
303
307
}
@@ -330,36 +334,25 @@ func datumToProtoValue(d tree.Datum) (*changefeedpb.Value, error) {
330
334
return & changefeedpb.Value {Value : & changefeedpb.Value_TimestampValue {TimestampValue : ts }}, nil
331
335
case * tree.DBytes :
332
336
return & changefeedpb.Value {Value : & changefeedpb.Value_BytesValue {BytesValue : []byte (* v )}}, nil
333
-
334
337
case * tree.DGeography :
335
- geostr , err := geo .SpatialObjectToWKT (v .Geography .SpatialObject (), - 1 )
336
- if err != nil {
337
- return nil , err
338
- }
339
- return & changefeedpb.Value {Value : & changefeedpb.Value_StringValue {StringValue : string (geostr )}}, nil
338
+ ewkb := v .EWKB ()
339
+ return & changefeedpb.Value {Value : & changefeedpb.Value_BytesValue {BytesValue : ewkb }}, nil
340
340
case * tree.DGeometry :
341
- geostr , err := geo .SpatialObjectToWKT (v .Geometry .SpatialObject (), - 1 )
342
- if err != nil {
343
- return nil , err
344
- }
345
- return & changefeedpb.Value {Value : & changefeedpb.Value_StringValue {StringValue : string (geostr )}}, nil
341
+ ewkb := v .EWKB ()
342
+ return & changefeedpb.Value {Value : & changefeedpb.Value_BytesValue {BytesValue : ewkb }}, nil
346
343
case * tree.DVoid :
347
344
return nil , nil
345
+ case * tree.DOid , * tree.DIPAddr , * tree.DBitArray , * tree.DBox2D ,
346
+ * tree.DTSVector , * tree.DTSQuery , * tree.DPGLSN , * tree.DPGVector :
347
+ return & changefeedpb.Value {Value : & changefeedpb.Value_StringValue {StringValue : tree .AsStringWithFlags (v , tree .FmtBareStrings , tree .FmtDataConversionConfig (dcc ), tree .FmtLocation (loc ))}}, nil
348
348
case * tree.DDate :
349
- date , err := v .ToTime ()
350
- if err != nil {
351
- return nil , err
352
- }
353
- return & changefeedpb.Value {Value : & changefeedpb.Value_DateValue {DateValue : date .Format ("2006-01-02" )}}, nil
349
+ return & changefeedpb.Value {Value : & changefeedpb.Value_DateValue {DateValue : tree .AsStringWithFlags (v , tree .FmtBareStrings , tree .FmtDataConversionConfig (dcc ), tree .FmtLocation (loc ))}}, nil
354
350
case * tree.DInterval :
355
- return & changefeedpb.Value {Value : & changefeedpb.Value_IntervalValue {IntervalValue : v . Duration . String ( )}}, nil
351
+ return & changefeedpb.Value {Value : & changefeedpb.Value_IntervalValue {IntervalValue : tree . AsStringWithFlags ( v , tree . FmtBareStrings , tree . FmtDataConversionConfig ( dcc ), tree . FmtLocation ( loc ) )}}, nil
356
352
case * tree.DUuid :
357
- return & changefeedpb.Value {Value : & changefeedpb.Value_UuidValue {UuidValue : strings . Trim ( v . UUID . String ( ), "'" )}}, nil
353
+ return & changefeedpb.Value {Value : & changefeedpb.Value_UuidValue {UuidValue : tree . AsStringWithFlags ( v , tree . FmtBareStrings , tree . FmtDataConversionConfig ( dcc ), tree . FmtLocation ( loc ) )}}, nil
358
354
case * tree.DTime , * tree.DTimeTZ :
359
- return & changefeedpb.Value {Value : & changefeedpb.Value_TimeValue {TimeValue : tree .AsStringWithFlags (v , tree .FmtBareStrings )}}, nil
360
- case * tree.DOid , * tree.DIPAddr , * tree.DBitArray , * tree.DBox2D ,
361
- * tree.DTSVector , * tree.DTSQuery , * tree.DPGLSN , * tree.DPGVector :
362
- return & changefeedpb.Value {Value : & changefeedpb.Value_StringValue {StringValue : d .String ()}}, nil
355
+ return & changefeedpb.Value {Value : & changefeedpb.Value_TimeValue {TimeValue : tree .AsStringWithFlags (v , tree .FmtBareStrings , tree .FmtDataConversionConfig (dcc ), tree .FmtLocation (loc ))}}, nil
363
356
default :
364
357
return nil , errors .AssertionFailedf ("unexpected type %T for datumToProtoValue" , d )
365
358
}
0 commit comments