github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/api/v2/unsafe.go (about)

     1  // Copyright 2022 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package v2
    15  
    16  import (
    17  	"context"
    18  	"net/http"
    19  	"strings"
    20  	"time"
    21  
    22  	"github.com/gin-gonic/gin"
    23  	"github.com/pingcap/errors"
    24  	tidbkv "github.com/pingcap/tidb/pkg/kv"
    25  	"github.com/pingcap/tiflow/cdc/kv"
    26  	"github.com/pingcap/tiflow/cdc/model"
    27  	cerror "github.com/pingcap/tiflow/pkg/errors"
    28  	"github.com/pingcap/tiflow/pkg/security"
    29  	"github.com/pingcap/tiflow/pkg/txnutil"
    30  	"github.com/pingcap/tiflow/pkg/txnutil/gc"
    31  	"github.com/tikv/client-go/v2/tikv"
    32  	pd "github.com/tikv/pd/client"
    33  )
    34  
    35  // CDCMetaData returns all etcd key values used by cdc
    36  func (h *OpenAPIV2) CDCMetaData(c *gin.Context) {
    37  	kvs, err := h.capture.GetEtcdClient().GetAllCDCInfo(c)
    38  	if err != nil {
    39  		_ = c.Error(err)
    40  		return
    41  	}
    42  	resp := make([]EtcdData, 0, len(kvs))
    43  	for _, pair := range kvs {
    44  		resp = append(resp, EtcdData{
    45  			Key:   string(pair.Key),
    46  			Value: string(pair.Value),
    47  		})
    48  	}
    49  	c.IndentedJSON(http.StatusOK, resp)
    50  }
    51  
    52  // ResolveLock resolve locks in regions
    53  func (h *OpenAPIV2) ResolveLock(c *gin.Context) {
    54  	var resolveLockReq ResolveLockReq
    55  	if err := c.BindJSON(&resolveLockReq); err != nil {
    56  		_ = c.Error(cerror.ErrAPIInvalidParam.Wrap(err))
    57  		return
    58  	}
    59  	var (
    60  		err       error
    61  		kvStorage tidbkv.Storage
    62  	)
    63  	if len(resolveLockReq.PDAddrs) > 0 {
    64  		kvStorage, err = kv.CreateTiStore(strings.Join(resolveLockReq.PDAddrs, ","),
    65  			&security.Credential{
    66  				CAPath:   resolveLockReq.CAPath,
    67  				CertPath: resolveLockReq.CertPath,
    68  				KeyPath:  resolveLockReq.KeyPath,
    69  			})
    70  		if err != nil {
    71  			_ = c.Error(err)
    72  			return
    73  		}
    74  	} else {
    75  		up, err := getCaptureDefaultUpstream(h.capture)
    76  		if err != nil {
    77  			_ = c.Error(err)
    78  			return
    79  		}
    80  		kvStorage = up.KVStorage
    81  	}
    82  
    83  	if kvStorage == nil {
    84  		c.Status(http.StatusServiceUnavailable)
    85  		return
    86  	}
    87  
    88  	txnResolver := txnutil.NewLockerResolver(kvStorage.(tikv.Storage),
    89  		// a fake changefeed id and namespace
    90  		model.ChangeFeedID{ID: "changefeed-client", Namespace: "default"})
    91  	err = txnResolver.Resolve(c, resolveLockReq.RegionID, resolveLockReq.Ts)
    92  	if err != nil {
    93  		_ = c.Error(err)
    94  		return
    95  	}
    96  	c.JSON(http.StatusOK, &EmptyResponse{})
    97  }
    98  
    99  // DeleteServiceGcSafePoint Delete CDC service GC safepoint in PD
   100  func (h *OpenAPIV2) DeleteServiceGcSafePoint(c *gin.Context) {
   101  	upstreamConfig := &UpstreamConfig{}
   102  	if err := c.BindJSON(upstreamConfig); err != nil {
   103  		_ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err))
   104  		return
   105  	}
   106  	err := h.withUpstreamConfig(c, upstreamConfig,
   107  		func(ctx context.Context, client pd.Client) error {
   108  			err := gc.RemoveServiceGCSafepoint(c, client,
   109  				h.capture.GetEtcdClient().GetGCServiceID())
   110  			if err != nil {
   111  				return cerror.WrapError(cerror.ErrInternalServerError, err)
   112  			}
   113  			return nil
   114  		})
   115  	if err != nil {
   116  		_ = c.Error(err)
   117  	}
   118  	c.JSON(http.StatusOK, &EmptyResponse{})
   119  }
   120  
   121  func (h *OpenAPIV2) withUpstreamConfig(c context.Context,
   122  	upstreamConfig *UpstreamConfig,
   123  	doWithClient func(ctx context.Context, client pd.Client) error,
   124  ) error {
   125  	var (
   126  		err      error
   127  		pdClient pd.Client
   128  	)
   129  	if upstreamConfig.ID > 0 {
   130  		upManager, err := h.capture.GetUpstreamManager()
   131  		if err != nil {
   132  			return errors.Trace(err)
   133  		}
   134  		up, ok := upManager.Get(upstreamConfig.ID)
   135  		if !ok {
   136  			return cerror.ErrUpstreamNotFound.GenWithStackByArgs(upstreamConfig.ID)
   137  		}
   138  		pdClient = up.PDClient
   139  	} else if len(upstreamConfig.PDAddrs) > 0 {
   140  		timeoutCtx, cancel := context.WithTimeout(c, 30*time.Second)
   141  		defer cancel()
   142  		pdClient, err = h.helpers.
   143  			getPDClient(timeoutCtx, upstreamConfig.PDAddrs, &security.Credential{
   144  				CAPath:        upstreamConfig.CAPath,
   145  				CertPath:      upstreamConfig.CertPath,
   146  				KeyPath:       upstreamConfig.KeyPath,
   147  				CertAllowedCN: nil,
   148  			})
   149  		if err != nil {
   150  			return cerror.WrapError(cerror.ErrInternalServerError, err)
   151  		}
   152  		defer pdClient.Close()
   153  	} else {
   154  		up, err := getCaptureDefaultUpstream(h.capture)
   155  		if err != nil {
   156  			return errors.Trace(err)
   157  		}
   158  		pdClient = up.PDClient
   159  	}
   160  	return doWithClient(c, pdClient)
   161  }