Skip to content

Commit

Permalink
meta, util: skip unmarshal unneeded JSON fields (#55304)
Browse files Browse the repository at this point in the history
ref #55324
  • Loading branch information
lance6716 authored Aug 13, 2024

Verified

This commit was created on GitHub.com and signed with GitHub’s verified signature. The key has expired.
1 parent 1f40cb0 commit 29d52a7
Showing 6 changed files with 376 additions and 10 deletions.
1 change: 1 addition & 0 deletions pkg/meta/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -18,6 +18,7 @@ go_library(
"//pkg/structure",
"//pkg/util/dbterror",
"//pkg/util/hack",
"//pkg/util/partialjson",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_kvproto//pkg/kvrpcpb",
"@com_github_pingcap_kvproto//pkg/resource_manager",
54 changes: 50 additions & 4 deletions pkg/meta/meta.go
Original file line number Diff line number Diff line change
@@ -38,6 +38,7 @@ import (
"github.com/pingcap/tidb/pkg/structure"
"github.com/pingcap/tidb/pkg/util/dbterror"
"github.com/pingcap/tidb/pkg/util/hack"
"github.com/pingcap/tidb/pkg/util/partialjson"
)

var (
@@ -1136,10 +1137,9 @@ func (m *Meta) ListSimpleTables(dbID int64) ([]*model.TableNameInfo, error) {
continue
}

tbInfo := &model.TableNameInfo{}
err = json.Unmarshal(r.Value, tbInfo)
if err != nil {
return nil, errors.Trace(err)
tbInfo, err2 := FastUnmarshalTableNameInfo(r.Value)
if err2 != nil {
return nil, errors.Trace(err2)
}

tables = append(tables, tbInfo)
@@ -1148,6 +1148,52 @@ func (m *Meta) ListSimpleTables(dbID int64) ([]*model.TableNameInfo, error) {
return tables, nil
}

var tableNameInfoFields = []string{"id", "name"}

// FastUnmarshalTableNameInfo is exported for testing.
func FastUnmarshalTableNameInfo(data []byte) (*model.TableNameInfo, error) {
m, err := partialjson.ExtractTopLevelMembers(data, tableNameInfoFields)
if err != nil {
return nil, errors.Trace(err)
}

idTokens, ok := m["id"]
if !ok {
return nil, errors.New("id field not found in JSON")
}
if len(idTokens) != 1 {
return nil, errors.Errorf("unexpected id field in JSON, %v", idTokens)
}
num, ok := idTokens[0].(json.Number)
if !ok {
return nil, errors.Errorf(
"id field is not a number, got %T %v", idTokens[0], idTokens[0],
)
}
id, err := num.Int64()
if err != nil {
return nil, errors.Trace(err)
}

nameTokens, ok := m["name"]
if !ok {
return nil, errors.New("name field not found in JSON")
}
// 6 tokens; {, O, ..., L, ..., }
if len(nameTokens) != 6 {
return nil, errors.Errorf("unexpected name field in JSON, %v", nameTokens)
}
name, ok := nameTokens[2].(string)
if !ok {
return nil, errors.Errorf("unexpected name field in JSON, %v", nameTokens)
}

return &model.TableNameInfo{
ID: id,
Name: model.NewCIStr(name),
}, nil
}

// ListDatabases shows all databases.
func (m *Meta) ListDatabases() ([]*model.DBInfo, error) {
res, err := m.txn.HGetAll(mDBs)
54 changes: 48 additions & 6 deletions pkg/meta/meta_test.go
Original file line number Diff line number Diff line change
@@ -765,10 +765,9 @@ func TestTableNameExtract(t *testing.T) {
require.Equal(t, `"\"啊"`, meta.Unescape(nameLMatch[1]))
}

func BenchmarkIsTableInfoMustLoad(b *testing.B) {
benchCases := [][2]string{
{"narrow", `CREATE TABLE t (c INT PRIMARY KEY);`},
{"wide", `
var benchCases = [][2]string{
{"narrow", `CREATE TABLE t (c INT PRIMARY KEY);`},
{"wide", `
CREATE TABLE t (
c BIGINT PRIMARY KEY AUTO_RANDOM,
c2 TINYINT,
@@ -789,16 +788,17 @@ CREATE TABLE t (
UNIQUE INDEX idx4(c12),
INDEX idx5((c + c2))
);`},
}
}

func BenchmarkIsTableInfoMustLoad(b *testing.B) {
for _, benchCase := range benchCases {
b.Run(benchCase[0], func(b *testing.B) {
benchIsTableInfoMustLoad(b, benchCase[1])
})
}
}

func benchIsTableInfoMustLoad(b *testing.B, sql string) {
func getTableInfoJSON(b *testing.B, sql string) []byte {
p := parser.New()
stmt, err := p.ParseOneStmt(sql, "", "")
require.NoError(b, err)
@@ -808,9 +808,51 @@ func benchIsTableInfoMustLoad(b *testing.B, sql string) {
data, err := json.Marshal(tblInfo)
require.NoError(b, err)

return data
}

func benchIsTableInfoMustLoad(b *testing.B, sql string) {
data := getTableInfoJSON(b, sql)

b.ResetTimer()
for i := 0; i < b.N; i++ {
got := meta.IsTableInfoMustLoad(data)
intest.Assert(!got)
}
}

func BenchmarkTableNameInfo(b *testing.B) {
for _, benchCase := range benchCases {
b.Run(benchCase[0]+"-json", func(b *testing.B) {
benchJSONTableNameInfo(b, benchCase[1])
})
b.Run(benchCase[0]+"-fastjson", func(b *testing.B) {
benchFastJSONTableNameInfo(b, benchCase[1])
})
}
}

func benchJSONTableNameInfo(b *testing.B, sql string) {
data := getTableInfoJSON(b, sql)

b.ResetTimer()
for i := 0; i < b.N; i++ {
tbInfo := &model.TableNameInfo{}
err := json.Unmarshal(data, tbInfo)
intest.Assert(tbInfo.ID == 1)
intest.Assert(tbInfo.Name.L == "t")
intest.AssertNoError(err)
}
}

func benchFastJSONTableNameInfo(b *testing.B, sql string) {
data := getTableInfoJSON(b, sql)

b.ResetTimer()
for i := 0; i < b.N; i++ {
tbInfo, err := meta.FastUnmarshalTableNameInfo(data)
intest.AssertNoError(err)
intest.Assert(tbInfo.ID == 1)
intest.Assert(tbInfo.Name.L == "t")
}
}
31 changes: 31 additions & 0 deletions pkg/util/partialjson/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "fastjson",
srcs = ["extract.go"],
importpath = "github.com/pingcap/tidb/pkg/util/fastjson",
visibility = ["//visibility:public"],
)

go_test(
name = "fastjson_test",
timeout = "short",
srcs = ["extract_test.go"],
embed = [":fastjson"],
flaky = True,
deps = ["@com_github_stretchr_testify//require"],
)

go_library(
name = "partialjson",
srcs = ["extract.go"],
importpath = "github.com/pingcap/tidb/pkg/util/partialjson",
visibility = ["//visibility:public"],
)

go_test(
name = "partialjson_test",
srcs = ["extract_test.go"],
embed = [":partialjson"],
deps = ["@com_github_stretchr_testify//require"],
)
170 changes: 170 additions & 0 deletions pkg/util/partialjson/extract.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,170 @@
// Copyright 2024 PingCAP, Inc.
//
// 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.

package partialjson

import (
"bytes"
"encoding/json"
"fmt"
"io"
)

type topLevelJSONTokenIter struct {
d *json.Decoder
level int
}

func newTopLevelJSONTokenIter(content []byte) *topLevelJSONTokenIter {
d := json.NewDecoder(bytes.NewReader(content))
d.UseNumber()
return &topLevelJSONTokenIter{
d: d,
}
}

func unexpectedEOF(err error) error {
if err == io.EOF {
return io.ErrUnexpectedEOF
}
return err
}

// readName reads a name belongs to the top-level of JSON objects. Caller should
// call readOrDiscardValue to consume its value before calling next readName.
func (i *topLevelJSONTokenIter) readName() (string, error) {
ts, err := i.next(false)
if err != nil {
return "", err
}
if len(ts) != 1 {
return "", fmt.Errorf("unexpected JSON name, %v", ts)
}
name, ok := ts[0].(string)
if !ok {
// > An object is an unordered collection of zero or more name/value
// pairs, where a name is a string...
// https://datatracker.ietf.org/doc/html/rfc8259#section-1
return "", fmt.Errorf("unexpected JSON name, %T %v", ts, ts)
}
return name, nil
}

// readOrDiscardValue reads a value belongs to the top-level of JSON objects. It
// must be called after readName. If caller don't need the value, it can pass
// true to discard it.
func (i *topLevelJSONTokenIter) readOrDiscardValue(discard bool) ([]json.Token, error) {
return i.next(discard)
}

// next is an internal method to iterate the JSON tokens. Callers should use
// readName / readOrDiscardValue instead.
func (i *topLevelJSONTokenIter) next(discard bool) ([]json.Token, error) {
if i.level == 0 {
t, err := i.d.Token()
if err != nil {
return nil, err
}

if t != json.Delim('{') {
return nil, fmt.Errorf(
"expected '{' for topLevelJSONTokenIter, got %T %v",
t, t,
)
}
i.level++
}

var longValue []json.Token

if i.level == 1 {
t, err := i.d.Token()
if err != nil {
return nil, unexpectedEOF(err)
}
delim, ok := t.(json.Delim)
if !ok {
return []json.Token{t}, nil
}

switch delim {
case '}', ']':
// we are at top level and now exit this level, which means the content is end.
i.level--
return nil, io.EOF
case '{', '[':
i.level++
// go to below loop to consume this level
if !discard {
longValue = make([]json.Token, 0, 16)
longValue = append(longValue, t)
}
}
}

for i.level > 1 {
t, err := i.d.Token()
if err != nil {
return nil, unexpectedEOF(err)
}
if !discard {
longValue = append(longValue, t)
}

delim, ok := t.(json.Delim)
if !ok {
continue
}

switch delim {
case '{', '[':
i.level++
case '}', ']':
i.level--
}
}
return longValue, nil
}

// ExtractTopLevelMembers extracts tokens of given top level members from a JSON
// text. It will stop parsing when all keys are found.
func ExtractTopLevelMembers(content []byte, names []string) (map[string][]json.Token, error) {
remainNames := make(map[string]struct{}, len(names))
for _, k := range names {
remainNames[k] = struct{}{}
}
ret := make(map[string][]json.Token, len(names))
iter := newTopLevelJSONTokenIter(content)
for len(remainNames) > 0 {
name, err := iter.readName()
if err != nil {
return nil, err
}
_, ok := remainNames[name]
if ok {
val, err2 := iter.readOrDiscardValue(false)
if err2 != nil {
return nil, err2
}
ret[name] = val
delete(remainNames, name)
} else {
_, err2 := iter.readOrDiscardValue(true)
if err2 != nil {
return nil, err2
}
}
}
return ret, nil
}
Loading

0 comments on commit 29d52a7

Please sign in to comment.