Skip to content

Commit

Permalink
Merge #52169
Browse files Browse the repository at this point in the history
52169: sql: Implement crdb_internal.pb_to_json function r=miretskiy a=miretskiy

Informs #47534

`crdb_internal.pb_to_json` function can convert any protocol
message linked into our binary to an JSONB object.

The function take 2 arguments: the fully qualified name of the
protocol message, and the encoded bytes for that message.
The returned JSONB object can then be queried, using regular JSONB
primitives.

```
SELECT  crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', descriptor)->'Union'->'Table'
FROM system.descriptor where id=15;
```

Release Notes: None

Co-authored-by: Yevgeniy Miretskiy <yevgeniy@cockroachlabs.com>
  • Loading branch information
craig[bot] and Yevgeniy Miretskiy committed Aug 1, 2020
2 parents 229b1bb + 2579d93 commit 2cddfdc
Show file tree
Hide file tree
Showing 6 changed files with 280 additions and 0 deletions.
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -1620,6 +1620,8 @@ Negative azimuth values and values greater than 2π (360 degrees) are supported.
</span></td></tr>
<tr><td><a name="array_to_json"></a><code>array_to_json(array: anyelement[], pretty_bool: <a href="bool.html">bool</a>) &rarr; jsonb</code></td><td><span class="funcdesc"><p>Returns the array as JSON or JSONB.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.pb_to_json"></a><code>crdb_internal.pb_to_json(pbname: <a href="string.html">string</a>, data: <a href="bytes.html">bytes</a>) &rarr; jsonb</code></td><td><span class="funcdesc"><p>Converts protocol message to its JSONB representation.</p>
</span></td></tr>
<tr><td><a name="json_array_length"></a><code>json_array_length(json: jsonb) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>Returns the number of elements in the outermost JSON or JSONB array.</p>
</span></td></tr>
<tr><td><a name="json_build_array"></a><code>json_build_array(anyelement...) &rarr; jsonb</code></td><td><span class="funcdesc"><p>Builds a possibly-heterogeneously-typed JSON or JSONB array out of a variadic argument list.</p>
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/builtin_function
Original file line number Diff line number Diff line change
Expand Up @@ -2771,3 +2771,12 @@ SELECT num_nulls(a, b), num_nonnulls(a, b) FROM nulls_test
1 1
1 1
2 0

query T
select crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', descriptor)->'database'->>'name' from system.descriptor where id=1
----
system


query error pq: crdb_internal.pb_to_json\(\): unknown proto message type cockroach.sql.sqlbase.descriptor
select crdb_internal.pb_to_json('cockroach.sql.sqlbase.descriptor', descriptor)->'database'->>'name' from system.descriptor where id=1
26 changes: 26 additions & 0 deletions pkg/sql/protoreflect/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package protoreflect

import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

func TestMain(m *testing.M) {
security.SetAssetLoader(securitytest.EmbeddedAssets)
randutil.SeedForTests()
os.Exit(m.Run())
}
86 changes: 86 additions & 0 deletions pkg/sql/protoreflect/utils.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package protoreflect

import (
"encoding/json"
"reflect"

jsonb "github.com/cockroachdb/cockroach/pkg/util/json"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
"github.com/gogo/protobuf/jsonpb"
"github.com/gogo/protobuf/proto"
)

// DecodeMessage decodes protocol message specified as its fully
// qualified name, and it's marshaled data, into a protoutil.Message.
func DecodeMessage(name string, data []byte) (protoutil.Message, error) {
// Get the reflected type of the protocol message.
rt := proto.MessageType(name)
if rt == nil {
return nil, errors.Newf("unknown proto message type %s", name)
}

// If the message is known, we should get the pointer to our message.
if rt.Kind() != reflect.Ptr {
return nil, errors.AssertionFailedf(
"expected ptr to message, got %s instead", rt.Kind().String())
}

// Construct message of appropriate type, through reflection.
rt = rt.Elem()
rv := reflect.New(rt)

msg, ok := rv.Interface().(protoutil.Message)

if !ok {
// Just to be safe;
return nil, errors.AssertionFailedf(
"unexpected proto type for %s; expected protoutil.Message, got %T",
name, rv.Interface())
}

// Now, parse data as our proto message.
if err := protoutil.Unmarshal(data, msg); err != nil {
return nil, errors.Wrapf(err, "failed to unmarshal proto %s", name)
}
return msg, nil
}

// MessageToJSON converts a protocol message into a JSONB object.
func MessageToJSON(msg protoutil.Message) (jsonb.JSON, error) {
// Convert to json.
jsonEncoder := jsonpb.Marshaler{}
msgJSON, err := jsonEncoder.MarshalToString(msg)
if err != nil {
return nil, errors.Wrapf(err, "marshaling %s; msg=%+v", proto.MessageName(msg), msg)
}

// We need to take encoded json object and, unfortunately, unmarshal
// it again, this time as a string->value map, so that we can construct
// JSONB object to return.
var msgMap map[string]interface{}
if err := json.Unmarshal([]byte(msgJSON), &msgMap); err != nil {
return nil, errors.Wrap(err, "unmarshaling to map")
}

// Build JSONB.
builder := jsonb.NewObjectBuilder(len(msgMap))
for k, v := range msgMap {
jv, err := jsonb.MakeJSON(v)
if err != nil {
return nil, errors.Wrapf(err, "encoding json value for %s", k)
}
builder.Add(k, jv)
}
return builder.Build(), nil
}
130 changes: 130 additions & 0 deletions pkg/sql/protoreflect/utils_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package protoreflect

import (
"reflect"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/geo/geoindex"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/gogo/protobuf/jsonpb"
pbtypes "github.com/gogo/protobuf/types"
"github.com/stretchr/testify/require"
)

func makeAny(t *testing.T, msg protoutil.Message) *pbtypes.Any {
any, err := pbtypes.MarshalAny(msg)
require.NoError(t, err)
return any
}

func TestMessageToJSONBRoundTrip(t *testing.T) {
defer leaktest.AfterTest(t)()

for _, tc := range []struct {
pbname string
message protoutil.Message
}{
{ // Just a simple Message
pbname: "cockroach.sql.sqlbase.Descriptor",
message: &sqlbase.Descriptor{
Union: &sqlbase.Descriptor_Table{
Table: &sqlbase.TableDescriptor{Name: "the table"},
},
},
},
{ // Message with an array
pbname: "cockroach.sql.sqlbase.ColumnDescriptor",
message: &sqlbase.ColumnDescriptor{
Name: "column",
ID: 123,
OwnsSequenceIds: []sqlbase.ID{3, 2, 1},
},
},
{ // Message with an array and other embedded descriptors
pbname: "cockroach.sql.sqlbase.IndexDescriptor",
message: &sqlbase.IndexDescriptor{
Name: "myidx",
ID: 42,
Unique: true,
ColumnNames: []string{"foo", "bar", "buz"},
ColumnDirections: []sqlbase.IndexDescriptor_Direction{sqlbase.IndexDescriptor_ASC},
GeoConfig: geoindex.Config{
S2Geography: &geoindex.S2GeographyConfig{S2Config: &geoindex.S2Config{
MinLevel: 123,
MaxLevel: 321,
}},
S2Geometry: &geoindex.S2GeometryConfig{
MinX: 567,
MaxX: 765,
},
},
},
},
{ // Message with embedded google.protobuf.Any message;
// nested inside other message; with maps
pbname: "cockroach.util.tracing.RecordedSpan",
message: &tracing.RecordedSpan{
TraceID: 123,
Tags: map[string]string{"one": "1", "two": "2", "three": "3"},
Stats: makeAny(t, &sqlbase.ColumnDescriptor{Name: "bogus stats"}),
},
},
{ // Message deeply nested inside other message
pbname: "cockroach.sql.sqlbase.TableDescriptor.SequenceOpts.SequenceOwner",
message: &sqlbase.TableDescriptor_SequenceOpts_SequenceOwner{OwnerColumnID: 123},
},
} {
t.Run(tc.pbname, func(t *testing.T) {
protoData, err := protoutil.Marshal(tc.message)
require.NoError(t, err)

// Decode proto bytes to message and compare.
decoded, err := DecodeMessage(tc.pbname, protoData)
require.NoError(t, err)
require.Equal(t, tc.message, decoded)

// Encode message as json
jsonb, err := MessageToJSON(decoded)
require.NoError(t, err)

// Recreate message from json
fromJSON := reflect.New(reflect.TypeOf(tc.message).Elem()).Interface().(protoutil.Message)

json := &jsonpb.Unmarshaler{}
require.NoError(t, json.Unmarshal(strings.NewReader(jsonb.String()), fromJSON))

require.Equal(t, tc.message, fromJSON)
})
}
}

// Ensure we don't blow up when asking to convert invalid
// data.
func TestInvalidConversions(t *testing.T) {
defer leaktest.AfterTest(t)()

t.Run("no such messagge", func(t *testing.T) {
_, err := DecodeMessage("no.such.message", nil)
require.Error(t, err)
})

t.Run("must be message type", func(t *testing.T) {
// Valid proto enum, but we require types.
_, err := DecodeMessage("cockroach.sql.sqlbase.SystemColumnKind", nil)
require.Error(t, err)
})
}
27 changes: 27 additions & 0 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/protoreflect"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
Expand Down Expand Up @@ -2864,6 +2865,32 @@ may increase either contention or retry errors, or both.`,

"jsonb_array_length": makeBuiltin(jsonProps(), jsonArrayLengthImpl),

"crdb_internal.pb_to_json": makeBuiltin(
jsonProps(),
tree.Overload{
Types: tree.ArgTypes{
{"pbname", types.String},
{"data", types.Bytes},
},
ReturnType: tree.FixedReturnType(types.Jsonb),
Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
msg, err := protoreflect.DecodeMessage(
string(tree.MustBeDString(args[0])),
[]byte(tree.MustBeDBytes(args[1])),
)
if err != nil {
return nil, err
}
j, err := protoreflect.MessageToJSON(msg)
if err != nil {
return nil, err
}
return tree.NewDJSON(j), nil
},
Info: "Converts protocol message to its JSONB representation.",
Volatility: tree.VolatilityImmutable,
}),

// Enum functions.
"enum_first": makeBuiltin(
tree.FunctionProperties{NullableArgs: true, Category: categoryEnum},
Expand Down

0 comments on commit 2cddfdc

Please sign in to comment.