From eff49702174b322ef98dbec51861af3f4a9b5088 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 26 Jul 2023 13:54:03 +0800 Subject: [PATCH] server: split ttl handler into new package (#45569) ref pingcap/tidb#44940 --- server/BUILD.bazel | 2 +- server/handler/BUILD.bazel | 12 +++++ server/handler/ttlhandler/BUILD.bazel | 19 +++++++ server/handler/ttlhandler/ttl.go | 73 +++++++++++++++++++++++++++ server/handler/util.go | 73 +++++++++++++++++++++++++++ server/http_handler.go | 36 ------------- server/http_status.go | 3 +- 7 files changed, 180 insertions(+), 38 deletions(-) create mode 100644 server/handler/BUILD.bazel create mode 100644 server/handler/ttlhandler/BUILD.bazel create mode 100644 server/handler/ttlhandler/ttl.go create mode 100644 server/handler/util.go diff --git a/server/BUILD.bazel b/server/BUILD.bazel index 4fb553fb121a7..19b852e5bb37b 100644 --- a/server/BUILD.bazel +++ b/server/BUILD.bazel @@ -51,6 +51,7 @@ go_library( "//privilege/privileges", "//privilege/privileges/ldap", "//server/err", + "//server/handler/ttlhandler", "//server/internal", "//server/internal/column", "//server/internal/dump", @@ -75,7 +76,6 @@ go_library( "//table", "//table/tables", "//tablecodec", - "//ttl/client", "//types", "//util", "//util/arena", diff --git a/server/handler/BUILD.bazel b/server/handler/BUILD.bazel new file mode 100644 index 0000000000000..05d49a9e36495 --- /dev/null +++ b/server/handler/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "handler", + srcs = ["util.go"], + importpath = "github.com/pingcap/tidb/server/handler", + visibility = ["//visibility:public"], + deps = [ + "//parser/terror", + "@com_github_pingcap_errors//:errors", + ], +) diff --git a/server/handler/ttlhandler/BUILD.bazel b/server/handler/ttlhandler/BUILD.bazel new file mode 100644 index 0000000000000..d7ccfbfeae123 --- /dev/null +++ b/server/handler/ttlhandler/BUILD.bazel @@ -0,0 +1,19 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "ttlhandler", + srcs = ["ttl.go"], + importpath = "github.com/pingcap/tidb/server/handler/ttlhandler", + visibility = ["//visibility:public"], + deps = [ + "//kv", + "//server/handler", + "//session", + "//ttl/client", + "//util/logutil", + "@com_github_gorilla_mux//:mux", + "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_log//:log", + "@org_uber_go_zap//:zap", + ], +) diff --git a/server/handler/ttlhandler/ttl.go b/server/handler/ttlhandler/ttl.go new file mode 100644 index 0000000000000..daf8e7bf058f7 --- /dev/null +++ b/server/handler/ttlhandler/ttl.go @@ -0,0 +1,73 @@ +// 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 ttlhandler + +import ( + "net/http" + "strings" + + "github.com/gorilla/mux" + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/server/handler" + "github.com/pingcap/tidb/session" + ttlcient "github.com/pingcap/tidb/ttl/client" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +// TTLJobTriggerHandler is used to trigger a TTL job manually +type TTLJobTriggerHandler struct { + store kv.Storage +} + +// NewTTLJobTriggerHandler returns a new TTLJobTriggerHandler +func NewTTLJobTriggerHandler(store kv.Storage) *TTLJobTriggerHandler { + return &TTLJobTriggerHandler{store: store} +} + +// ServeHTTP handles request of triger a ttl job +func (h TTLJobTriggerHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + if req.Method != http.MethodPost { + handler.WriteError(w, errors.Errorf("This api only support POST method")) + return + } + + params := mux.Vars(req) + dbName := strings.ToLower(params["db"]) + tableName := strings.ToLower(params["table"]) + + ctx := req.Context() + dom, err := session.GetDomain(h.store) + if err != nil { + log.Error("failed to get session domain", zap.Error(err)) + handler.WriteError(w, err) + return + } + + cli := dom.TTLJobManager().GetCommandCli() + resp, err := ttlcient.TriggerNewTTLJob(ctx, cli, dbName, tableName) + if err != nil { + log.Error("failed to trigger new TTL job", zap.Error(err)) + handler.WriteError(w, err) + return + } + handler.WriteData(w, resp) + logutil.Logger(ctx).Info("trigger TTL job manually successfully", + zap.String("dbName", dbName), + zap.String("tableName", tableName), + zap.Any("response", resp)) +} diff --git a/server/handler/util.go b/server/handler/util.go new file mode 100644 index 0000000000000..9039eed667a21 --- /dev/null +++ b/server/handler/util.go @@ -0,0 +1,73 @@ +// 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 handler + +import ( + "encoding/json" + "net/http" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/terror" +) + +//revive:disable +const ( + DBName = "db" + HexKey = "hexKey" + IndexName = "index" + Handle = "handle" + RegionID = "regionID" + StartTS = "startTS" + TableName = "table" + TableID = "tableID" + ColumnID = "colID" + ColumnTp = "colTp" + ColumnFlag = "colFlag" + ColumnLen = "colLen" + RowBin = "rowBin" + Snapshot = "snapshot" + FileName = "filename" + DumpPartitionStats = "dumpPartitionStats" + Begin = "begin" + End = "end" +) + +const ( + HeaderContentType = "Content-Type" + ContentTypeJSON = "application/json" +) + +//revive:enable + +// WriteError writes error to response. +func WriteError(w http.ResponseWriter, err error) { + w.WriteHeader(http.StatusBadRequest) + _, err = w.Write([]byte(err.Error())) + terror.Log(errors.Trace(err)) +} + +// WriteData writes data to response. +func WriteData(w http.ResponseWriter, data interface{}) { + js, err := json.MarshalIndent(data, "", " ") + if err != nil { + WriteError(w, err) + return + } + // write response + w.Header().Set(HeaderContentType, ContentTypeJSON) + w.WriteHeader(http.StatusOK) + _, err = w.Write(js) + terror.Log(errors.Trace(err)) +} diff --git a/server/http_handler.go b/server/http_handler.go index 86a5c9c2fe2e5..cb4b0666a285e 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -58,7 +58,6 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" - ttlcient "github.com/pingcap/tidb/ttl/client" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/codec" @@ -2235,38 +2234,3 @@ func (h labelHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { writeData(w, config.GetGlobalConfig().Labels) } - -// ttlJobTriggerHandler is used to trigger a TTL job manually -type ttlJobTriggerHandler struct { - store kv.Storage -} - -// ServeHTTP handles request of triger a ttl job -func (h ttlJobTriggerHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { - if req.Method != http.MethodPost { - writeError(w, errors.Errorf("This api only support POST method")) - return - } - - params := mux.Vars(req) - dbName := strings.ToLower(params["db"]) - tableName := strings.ToLower(params["table"]) - - ctx := req.Context() - dom, err := session.GetDomain(h.store) - if err != nil { - log.Error("failed to get session domain", zap.Error(err)) - writeError(w, err) - return - } - - cli := dom.TTLJobManager().GetCommandCli() - resp, err := ttlcient.TriggerNewTTLJob(ctx, cli, dbName, tableName) - if err != nil { - log.Error("failed to trigger new TTL job", zap.Error(err)) - writeError(w, err) - return - } - writeData(w, resp) - logutil.Logger(ctx).Info("trigger TTL job manually successfully", zap.String("dbName", dbName), zap.String("tableName", tableName), zap.Any("response", resp)) -} diff --git a/server/http_status.go b/server/http_status.go index e031d7d97a9ef..3aefe53da02e7 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/server/handler/ttlhandler" util2 "github.com/pingcap/tidb/server/internal/util" "github.com/pingcap/tidb/store" "github.com/pingcap/tidb/util" @@ -420,7 +421,7 @@ func (s *Server) startHTTPServer() { router.Handle("/test/ddl/hook", &ddlHookHandler{tikvHandlerTool.Store.(kv.Storage)}) // ttlJobTriggerHandler is enabled only for tests, so we can accelerate the schedule of TTL job - router.Handle("/test/ttl/trigger/{db}/{table}", &ttlJobTriggerHandler{tikvHandlerTool.Store.(kv.Storage)}) + router.Handle("/test/ttl/trigger/{db}/{table}", ttlhandler.NewTTLJobTriggerHandler(tikvHandlerTool.Store.(kv.Storage))) var ( httpRouterPage bytes.Buffer