Skip to content

Commit

Permalink
ddl,util: modify the collation of the column of multi-valued index to…
Browse files Browse the repository at this point in the history
… "binary" (pingcap#46993)

close pingcap#46717
  • Loading branch information
YangKeao authored and yibin87 committed Oct 31, 2023
1 parent 86d401c commit 7fd4553
Show file tree
Hide file tree
Showing 6 changed files with 128 additions and 0 deletions.
6 changes: 6 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -7158,6 +7158,12 @@ func BuildHiddenColumnInfo(ctx sessionctx.Context, indexPartSpecifications []*as
colInfo.FieldType.SetDecimal(types.MaxFsp)
}
}
// For an array, the collation is set to "binary". The collation has no effect on the array itself (as it's usually
// regarded as a JSON), but will influence how TiKV handles the index value.
if colInfo.FieldType.IsArray() {
colInfo.SetCharset("binary")
colInfo.SetCollate("binary")
}
checkDependencies := make(map[string]struct{})
for _, colName := range FindColumnNamesInExpr(idxPart.Expr) {
colInfo.Dependences[colName.Name.L] = struct{}{}
Expand Down
19 changes: 19 additions & 0 deletions ddl/tests/multivaluedindex/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "multivaluedindex_test",
timeout = "short",
srcs = [
"main_test.go",
"multi_valued_index_test.go",
],
flaky = True,
deps = [
"//infoschema",
"//parser/model",
"//testkit",
"//testkit/testsetup",
"@com_github_stretchr_testify//require",
"@org_uber_go_goleak//:goleak",
],
)
35 changes: 35 additions & 0 deletions ddl/tests/multivaluedindex/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
// Copyright 2023 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 multivaluedindex

import (
"testing"

"github.com/pingcap/tidb/testkit/testsetup"
"go.uber.org/goleak"
)

func TestMain(m *testing.M) {
testsetup.SetupForCommonTest()

opts := []goleak.Option{
goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"),
goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"),
goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"),
goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"),
}

goleak.VerifyTestMain(m, opts...)
}
47 changes: 47 additions & 0 deletions ddl/tests/multivaluedindex/multi_valued_index_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
// Copyright 2023 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 multivaluedindex

import (
"testing"

"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
)

func TestCreateMultiValuedIndexHasBinaryCollation(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)

tk.MustExec("create table test.t (pk varchar(4) primary key clustered, j json, str varchar(255), value int, key idx((cast(j as char(100) array)), str));")
is := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema)
require.NotNil(t, is)

tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)

foundIndex := false
for _, c := range tbl.Cols() {
if c.Hidden {
foundIndex = true
require.True(t, c.FieldType.IsArray())
require.Equal(t, c.FieldType.GetCharset(), "binary")
require.Equal(t, c.FieldType.GetCollate(), "binary")
}
}
require.True(t, foundIndex)
}
15 changes: 15 additions & 0 deletions planner/core/plan_to_pb_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,4 +79,19 @@ func TestColumnToProto(t *testing.T) {
}
pc = util.ColumnToProto(col2, false)
require.Len(t, pc.Elems, 2)

tp = types.NewFieldTypeBuilder().
SetType(mysql.TypeString).
SetCharset("utf8mb4").
SetCollate("utf8mb4_bin").
SetFlen(100).
SetFlag(10).
SetArray(true).
BuildP()
col3 := &model.ColumnInfo{
FieldType: *tp,
}
pc = util.ColumnToProto(col3, true)
expect = &tipb.ColumnInfo{ColumnId: 0, Tp: 0xfe, Collation: 63, ColumnLen: 100, Decimal: 0, Flag: 10, Elems: []string(nil), DefaultVal: []uint8(nil), PkHandle: false, XXX_unrecognized: []uint8(nil)}
require.Equal(t, expect, pc)
}
6 changes: 6 additions & 0 deletions util/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -423,6 +423,12 @@ func ColumnToProto(c *model.ColumnInfo, forIndex bool) *tipb.ColumnInfo {
if forIndex {
// Use array type for read the multi-valued index.
pc.Tp = int32(c.FieldType.ArrayType().GetType())
if c.FieldType.IsArray() {
// Use "binary" collation for read the multi-valued index. Most of the time, the `Collation` of this hidden
// column should already been set to "binary". However, in old versions, the collation is set to the default
// value. See https://github.com/pingcap/tidb/issues/46717
pc.Collation = int32(mysql.CollationNames["binary"])
}
} else {
pc.Tp = int32(c.GetType())
}
Expand Down

0 comments on commit 7fd4553

Please sign in to comment.