Skip to content

Commit

Permalink
server: split ttl handler into new package (#45569)
Browse files Browse the repository at this point in the history
ref #44940
  • Loading branch information
hawkingrei authored Jul 26, 2023
1 parent c0459da commit eff4970
Show file tree
Hide file tree
Showing 7 changed files with 180 additions and 38 deletions.
2 changes: 1 addition & 1 deletion server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ go_library(
"//privilege/privileges",
"//privilege/privileges/ldap",
"//server/err",
"//server/handler/ttlhandler",
"//server/internal",
"//server/internal/column",
"//server/internal/dump",
Expand All @@ -75,7 +76,6 @@ go_library(
"//table",
"//table/tables",
"//tablecodec",
"//ttl/client",
"//types",
"//util",
"//util/arena",
Expand Down
12 changes: 12 additions & 0 deletions server/handler/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -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",
],
)
19 changes: 19 additions & 0 deletions server/handler/ttlhandler/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_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",
],
)
73 changes: 73 additions & 0 deletions server/handler/ttlhandler/ttl.go
Original file line number Diff line number Diff line change
@@ -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))
}
73 changes: 73 additions & 0 deletions server/handler/util.go
Original file line number Diff line number Diff line change
@@ -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))
}
36 changes: 0 additions & 36 deletions server/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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))
}
3 changes: 2 additions & 1 deletion server/http_status.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down

0 comments on commit eff4970

Please sign in to comment.