Skip to content

Commit

Permalink
plugin: support dynamic enable/disable plugins (#11122) (#11157)
Browse files Browse the repository at this point in the history
  • Loading branch information
lysu authored and jackysp committed Jul 10, 2019
1 parent c96ecb4 commit 67efcb0
Show file tree
Hide file tree
Showing 14 changed files with 574 additions and 58 deletions.
52 changes: 52 additions & 0 deletions executor/admin_plugins.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
// Copyright 2019 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,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor

import (
"context"

"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/plugin"
"github.com/pingcap/tidb/util/chunk"
)

// AdminPluginsExec indicates AdminPlugins executor.
type AdminPluginsExec struct {
baseExecutor
Action core.AdminPluginsAction
Plugins []string
}

// Next implements the Executor Next interface.
func (e *AdminPluginsExec) Next(ctx context.Context, _ *chunk.Chunk) error {
switch e.Action {
case core.Enable:
return e.changeDisableFlagAndFlush(false)
case core.Disable:
return e.changeDisableFlagAndFlush(true)
}
return nil
}

func (e *AdminPluginsExec) changeDisableFlagAndFlush(disabled bool) error {
dom := domain.GetDomain(e.ctx)
for _, pluginName := range e.Plugins {
err := plugin.ChangeDisableFlagAndFlush(dom, pluginName, disabled)
if err != nil {
return err
}
}
return nil
}
6 changes: 6 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor {
return b.buildChecksumTable(v)
case *plannercore.ReloadExprPushdownBlacklist:
return b.buildReloadExprPushdownBlacklist(v)
case *plannercore.AdminPlugins:
return b.buildAdminPlugins(v)
case *plannercore.DDL:
return b.buildDDL(v)
case *plannercore.Deallocate:
Expand Down Expand Up @@ -467,6 +469,10 @@ func (b *executorBuilder) buildReloadExprPushdownBlacklist(v *plannercore.Reload
return &ReloadExprPushdownBlacklistExec{baseExecutor{ctx: b.ctx}}
}

func (b *executorBuilder) buildAdminPlugins(v *plannercore.AdminPlugins) Executor {
return &AdminPluginsExec{baseExecutor: baseExecutor{ctx: b.ctx}, Action: v.Action, Plugins: v.Plugins}
}

func (b *executorBuilder) buildDeallocate(v *plannercore.Deallocate) Executor {
base := newBaseExecutor(b.ctx, nil, v.ExplainID())
base.initCap = chunk.ZeroCapacity
Expand Down
2 changes: 1 addition & 1 deletion executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -1030,7 +1030,7 @@ func (e *ShowExec) fetchShowPlugins() error {
tiPlugins := plugin.GetAll()
for _, ps := range tiPlugins {
for _, p := range ps {
e.appendRow([]interface{}{p.Name, p.State.String(), p.Kind.String(), p.Path, p.License, strconv.Itoa(int(p.Version))})
e.appendRow([]interface{}{p.Name, p.StateValue(), p.Kind.String(), p.Path, p.License, strconv.Itoa(int(p.Version))})
}
}
return nil
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ require (
github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e
github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596
github.com/pingcap/parser v0.0.0-20190613045206-37cc370a20a4
github.com/pingcap/parser v0.0.0-20190710031629-52a9d3a79f41
github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669
github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible
github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -170,8 +170,8 @@ github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531/go.mod h1:QMdbTAXC
github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w=
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ=
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw=
github.com/pingcap/parser v0.0.0-20190613045206-37cc370a20a4 h1:r5BvCTM1R9U9EjJntFREb67GMsgn8IK9vLTQ/HzRZBc=
github.com/pingcap/parser v0.0.0-20190613045206-37cc370a20a4/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/parser v0.0.0-20190710031629-52a9d3a79f41 h1:hsCjAYfXliEMyRQTiNAYHyYATfURKNSK1J0eaKfOm1w=
github.com/pingcap/parser v0.0.0-20190710031629-52a9d3a79f41/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4=
github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM=
github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU=
Expand Down
17 changes: 17 additions & 0 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,23 @@ type ReloadExprPushdownBlacklist struct {
baseSchemaProducer
}

// AdminPluginsAction indicate action will be taken on plugins.
type AdminPluginsAction int

const (
// Enable indicates enable plugins.
Enable AdminPluginsAction = iota + 1
// Disable indicates disable plugins.
Disable
)

// AdminPlugins administrates tidb plugins.
type AdminPlugins struct {
baseSchemaProducer
Action AdminPluginsAction
Plugins []string
}

// Change represents a change plan.
type Change struct {
baseSchemaProducer
Expand Down
4 changes: 4 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -670,6 +670,10 @@ func (b *PlanBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) {
ret = p
case ast.AdminReloadExprPushdownBlacklist:
return &ReloadExprPushdownBlacklist{}, nil
case ast.AdminPluginEnable:
return &AdminPlugins{Action: Enable, Plugins: as.Plugins}, nil
case ast.AdminPluginDisable:
return &AdminPlugins{Action: Disable, Plugins: as.Plugins}, nil
default:
return nil, ErrUnsupportedType.GenWithStack("Unsupported ast.AdminStmt(%T) for buildAdmin", as)
}
Expand Down
3 changes: 1 addition & 2 deletions plugin/audit.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ package plugin
import (
"context"

"github.com/pingcap/parser/auth"
"github.com/pingcap/tidb/sessionctx/variable"
)

Expand Down Expand Up @@ -77,7 +76,7 @@ type AuditManifest struct {
Manifest
// OnConnectionEvent will be called when TiDB receive or disconnect from client.
// return error will ignore and close current connection.
OnConnectionEvent func(ctx context.Context, identity *auth.UserIdentity, event ConnectionEvent, info *variable.ConnectionInfo) error
OnConnectionEvent func(ctx context.Context, event ConnectionEvent, info *variable.ConnectionInfo) error
// OnGeneralEvent will be called during TiDB execution.
OnGeneralEvent func(ctx context.Context, sctx *variable.SessionVars, event GeneralEvent, cmd string)
// OnGlobalVariableEvent will be called when Change GlobalVariable.
Expand Down
42 changes: 42 additions & 0 deletions plugin/const_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
// Copyright 2019 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,
// See the License for the specific language governing permissions and
// limitations under the License.

package plugin

import (
"fmt"
"testing"
)

func TestConstToString(t *testing.T) {
kinds := map[fmt.Stringer]string{
Audit: "Audit",
Authentication: "Authentication",
Schema: "Schema",
Daemon: "Daemon",
Uninitialized: "Uninitialized",
Ready: "Ready",
Dying: "Dying",
Disable: "Disable",
Connected: "Connected",
Disconnect: "Disconnect",
ChangeUser: "ChangeUser",
PreAuth: "PreAuth",
ConnectionEvent(byte(15)): "",
}
for key, value := range kinds {
if key.String() != value {
t.Errorf("kind %s != %s", key.String(), kinds)
}
}
}
54 changes: 54 additions & 0 deletions plugin/helper_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
// Copyright 2019 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,
// See the License for the specific language governing permissions and
// limitations under the License.

package plugin

import "testing"

func TestPluginDeclare(t *testing.T) {
auditRaw := &AuditManifest{Manifest: Manifest{}}
auditExport := ExportManifest(auditRaw)
audit2 := DeclareAuditManifest(auditExport)
if audit2 != auditRaw {
t.Errorf("declare audit fail")
}

authRaw := &AuthenticationManifest{Manifest: Manifest{}}
authExport := ExportManifest(authRaw)
auth2 := DeclareAuthenticationManifest(authExport)
if auth2 != authRaw {
t.Errorf("declare auth fail")
}

schemaRaw := &SchemaManifest{Manifest: Manifest{}}
schemaExport := ExportManifest(schemaRaw)
schema2 := DeclareSchemaManifest(schemaExport)
if schema2 != schemaRaw {
t.Errorf("declare schema fail")
}

daemonRaw := &DaemonManifest{Manifest: Manifest{}}
daemonExport := ExportManifest(daemonRaw)
daemon2 := DeclareDaemonManifest(daemonExport)
if daemon2 != daemonRaw {
t.Errorf("declare daemon fail")
}
}

func TestDecode(t *testing.T) {
failID := ID("fail")
_, _, err := failID.Decode()
if err == nil {
t.Errorf("'fail' should not decode success")
}
}
Loading

0 comments on commit 67efcb0

Please sign in to comment.