Skip to content
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 7 additions & 1 deletion pkg/expression/constant_propagation.go
Original file line number Diff line number Diff line change
Expand Up @@ -425,7 +425,13 @@ func (s *propConstSolver) pickNewEQConds(visited []bool) (retMapper map[int]*Con
return nil
}
if updated {
retMapper[s.getColID(col)] = con
oriWarningCnt := s.ctx.GetEvalCtx().WarningCount()
newExpr := BuildCastFunction(s.ctx, con, col.GetType(s.ctx.GetEvalCtx()))
s.ctx.GetEvalCtx().TruncateWarnings(oriWarningCnt)

if newCon, ok := newExpr.(*Constant); ok {
retMapper[s.getColID(col)] = newCon
}
}
}
return
Expand Down
20 changes: 20 additions & 0 deletions pkg/expression/test/constantpropagation/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "constantpropagation_test",
timeout = "short",
srcs = [
"constant_propagation_test.go",
"main_test.go",
],
flaky = True,
deps = [
"//pkg/config",
"//pkg/testkit",
"//pkg/testkit/testmain",
"//pkg/testkit/testsetup",
"//pkg/util/timeutil",
"@com_github_tikv_client_go_v2//tikv",
"@org_uber_go_goleak//:goleak",
],
)
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
// Copyright 2022 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 constantpropagation

import (
"testing"

"github.com/pingcap/tidb/pkg/testkit"
)

func TestConstantPropagationMissingCastExpr(t *testing.T) {
store, _ := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table tl50cb7440 (" +
" col_43 decimal(30,30) not null," +
" primary key (col_43) /*t![clustered_index] clustered */," +
" unique key idx_12 (col_43)," +
" key idx_13 (col_43)," +
" unique key idx_14 (col_43)" +
") engine=innodb default charset=utf8 collate=utf8_bin;")
tk.MustExec("insert into tl50cb7440 values(0.000000000000000000000000000000),(0.400000000000000000000000000000);")
tk.MustQuery("with cte_8911 (col_47665) as" +
" (select mid(tl50cb7440.col_43, 6, 9) as r0" +
" from tl50cb7440" +
" where tl50cb7440.col_43 in (0, 0) and tl50cb7440.col_43 in (0))" +
" (select 1" +
" from cte_8911 where cte_8911.col_47665!='');").Check(testkit.Rows("1"))
}
57 changes: 57 additions & 0 deletions pkg/expression/test/constantpropagation/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
// 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 constantpropagation

import (
"testing"

"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/testkit/testmain"
"github.com/pingcap/tidb/pkg/testkit/testsetup"
"github.com/pingcap/tidb/pkg/util/timeutil"
"github.com/tikv/client-go/v2/tikv"
"go.uber.org/goleak"
)

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

config.UpdateGlobal(func(conf *config.Config) {
conf.TiKVClient.AsyncCommit.SafeWindow = 0
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0
conf.Experimental.AllowsExpressionIndex = true
})
tikv.EnableFailpoints()

// Some test depends on the values of timeutil.SystemLocation()
// If we don't SetSystemTZ() here, the value would change unpredictable.
// Affected by the order whether a testsuite runs before or after integration test.
// Note, SetSystemTZ() is a sync.Once operation.
timeutil.SetSystemTZ("system")

opts := []goleak.Option{
goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"),
goleak.IgnoreTopFunction("github.com/bazelbuild/rules_go/go/tools/bzltestutil.RegisterTimeoutHandler.func1"),
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.IgnoreTopFunction("github.com/pingcap/tidb/pkg/ttl/ttlworker.(*ttlScanWorker).loop"),
goleak.IgnoreTopFunction("github.com/pingcap/tidb/pkg/ttl/client.(*mockClient).WatchCommand.func1"),
goleak.IgnoreTopFunction("github.com/pingcap/tidb/pkg/ttl/ttlworker.(*JobManager).jobLoop"),
}

goleak.VerifyTestMain(m, opts...)
}