tso

package
v0.1.0 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Jun 15, 2023 License: Apache-2.0 Imports: 41 Imported by: 0

Documentation

Index

Constants

View Source
const (

	// UpdateTimestampGuard is the min timestamp interval.
	UpdateTimestampGuard = time.Millisecond

	// MaxSuffixBits indicates the max number of suffix bits.
	MaxSuffixBits = 4
)
View Source
const (
	// GlobalDCLocation is the Global TSO Allocator's DC location label.
	GlobalDCLocation = "global"
)

Variables

View Source
var (
	// PriorityCheck exported is only for test.
	PriorityCheck = time.Minute
)

Functions

func CalSuffixBits

func CalSuffixBits(maxSuffix int32) int

CalSuffixBits calculates the bits of suffix by the max suffix sign.

func FilterAvailableLeadership

func FilterAvailableLeadership() func(ag *allocatorGroup) bool

FilterAvailableLeadership will filter out the allocatorGroup whose leadership is available.

func FilterDCLocation

func FilterDCLocation(dcLocation string) func(ag *allocatorGroup) bool

FilterDCLocation will filter out the allocatorGroup with a given dcLocation.

func FilterUnavailableLeadership

func FilterUnavailableLeadership() func(ag *allocatorGroup) bool

FilterUnavailableLeadership will filter out the allocatorGroup whose leadership is unavailable.

func FilterUninitialized

func FilterUninitialized() func(ag *allocatorGroup) bool

FilterUninitialized will filter out the allocatorGroup uninitialized.

Types

type AdminHandler

type AdminHandler struct {
	// contains filtered or unexported fields
}

AdminHandler wrap the basic tso handler to provide http service.

func NewAdminHandler

func NewAdminHandler(handler Handler, rd *render.Render) *AdminHandler

NewAdminHandler returns a new admin handler.

func (*AdminHandler) ResetTS

func (h *AdminHandler) ResetTS(w http.ResponseWriter, r *http.Request)

ResetTS is the http.HandlerFunc of ResetTS FIXME: details of input json body params @Tags admin @Summary Reset the ts. @Accept json @Param body body object true "json params" @Produce json @Success 200 {string} string "Reset ts successfully." @Failure 400 {string} string "The input is invalid." @Failure 403 {string} string "Reset ts is forbidden." @Failure 500 {string} string "TSO server failed to proceed the request." @Router /admin/reset-ts [post] if force-use-larger=true:

reset ts to max(current ts, input ts).

else:

reset ts to input ts if it > current ts and < upper bound, error if not in that range

during EBS based restore, we call this to make sure ts of pd >= resolved_ts in backup.

type Allocator

type Allocator interface {
	// Initialize is used to initialize a TSO allocator.
	// It will synchronize TSO with etcd and initialize the
	// memory for later allocation work.
	Initialize(suffix int) error
	// IsInitialize is used to indicates whether this allocator is initialized.
	IsInitialize() bool
	// UpdateTSO is used to update the TSO in memory and the time window in etcd.
	UpdateTSO() error
	// SetTSO sets the physical part with given TSO. It's mainly used for BR restore.
	// Cannot set the TSO smaller than now in any case.
	// if ignoreSmaller=true, if input ts is smaller than current, ignore silently, else return error
	// if skipUpperBoundCheck=true, skip tso upper bound check
	SetTSO(tso uint64, ignoreSmaller, skipUpperBoundCheck bool) error
	// GenerateTSO is used to generate a given number of TSOs.
	// Make sure you have initialized the TSO allocator before calling.
	GenerateTSO(count uint32) (pdpb.Timestamp, error)
	// Reset is used to reset the TSO allocator.
	Reset()
}

Allocator is a Timestamp Oracle allocator.

func NewGlobalTSOAllocator

func NewGlobalTSOAllocator(
	ctx context.Context,
	am *AllocatorManager,
	startGlobalLeaderLoop bool,
) Allocator

NewGlobalTSOAllocator creates a new global TSO allocator.

func NewLocalTSOAllocator

func NewLocalTSOAllocator(
	am *AllocatorManager,
	leadership *election.Leadership,
	dcLocation string,
) Allocator

NewLocalTSOAllocator creates a new local TSO allocator.

type AllocatorGroupFilter

type AllocatorGroupFilter func(ag *allocatorGroup) bool

AllocatorGroupFilter is used to select AllocatorGroup.

type AllocatorManager

type AllocatorManager struct {
	// contains filtered or unexported fields
}

AllocatorManager is used to manage the TSO Allocators a PD server holds. It is in charge of maintaining TSO allocators' leadership, checking election priority, and forwarding TSO allocation requests to correct TSO Allocators.

func NewAllocatorManager

func NewAllocatorManager(
	ctx context.Context,
	keyspaceGroupID uint32,
	member ElectionMember,
	rootPath string,
	storage endpoint.TSOStorage,
	cfg Config,
	startGlobalLeaderLoop bool,
) *AllocatorManager

NewAllocatorManager creates a new TSO Allocator Manager.

func (*AllocatorManager) AllocatorDaemon

func (am *AllocatorManager) AllocatorDaemon(ctx context.Context)

AllocatorDaemon is used to update every allocator's TSO and check whether we have any new local allocator that needs to be set up.

func (*AllocatorManager) CleanUpDCLocation

func (am *AllocatorManager) CleanUpDCLocation() error

CleanUpDCLocation cleans up certain server's DCLocationInfo

func (*AllocatorManager) ClusterDCLocationChecker

func (am *AllocatorManager) ClusterDCLocationChecker()

ClusterDCLocationChecker collects all dc-locations of a cluster, computes some related info and stores them into the DCLocationInfo, then finally writes them into am.mu.clusterDCLocations.

func (*AllocatorManager) EnableLocalTSO

func (am *AllocatorManager) EnableLocalTSO() bool

EnableLocalTSO returns the value of AllocatorManager.enableLocalTSO.

func (*AllocatorManager) GetAllocator

func (am *AllocatorManager) GetAllocator(dcLocation string) (Allocator, error)

GetAllocator get the allocator by dc-location.

func (*AllocatorManager) GetAllocators

func (am *AllocatorManager) GetAllocators(filters ...AllocatorGroupFilter) []Allocator

GetAllocators get all allocators with some filters.

func (*AllocatorManager) GetClusterDCLocations

func (am *AllocatorManager) GetClusterDCLocations() map[string]DCLocationInfo

GetClusterDCLocations returns all dc-locations of a cluster with a copy of map, which satisfies dcLocation -> DCLocationInfo.

func (*AllocatorManager) GetClusterDCLocationsFromEtcd

func (am *AllocatorManager) GetClusterDCLocationsFromEtcd() (clusterDCLocations map[string][]uint64, err error)

GetClusterDCLocationsFromEtcd fetches dcLocation topology from etcd

func (*AllocatorManager) GetClusterDCLocationsNumber

func (am *AllocatorManager) GetClusterDCLocationsNumber() int

GetClusterDCLocationsNumber returns the number of cluster dc-locations.

func (*AllocatorManager) GetDCLocationInfo

func (am *AllocatorManager) GetDCLocationInfo(dcLocation string) (DCLocationInfo, bool)

GetDCLocationInfo returns a copy of DCLocationInfo of the given dc-location,

func (*AllocatorManager) GetHoldingLocalAllocatorLeaders

func (am *AllocatorManager) GetHoldingLocalAllocatorLeaders() ([]*LocalTSOAllocator, error)

GetHoldingLocalAllocatorLeaders returns all Local TSO Allocator leaders this server holds.

func (*AllocatorManager) GetLocalAllocatorLeaders

func (am *AllocatorManager) GetLocalAllocatorLeaders() (map[string]*pdpb.Member, error)

GetLocalAllocatorLeaders returns all Local TSO Allocator leaders' member info.

func (*AllocatorManager) GetLocalTSOSuffixPath

func (am *AllocatorManager) GetLocalTSOSuffixPath(dcLocation string) string

GetLocalTSOSuffixPath returns the etcd key of the Local TSO suffix for the given dc-location.

func (*AllocatorManager) GetLocalTSOSuffixPathPrefix

func (am *AllocatorManager) GetLocalTSOSuffixPathPrefix() string

GetLocalTSOSuffixPathPrefix returns the etcd key prefix of the Local TSO suffix for the given dc-location.

func (*AllocatorManager) GetMaxLocalTSO

func (am *AllocatorManager) GetMaxLocalTSO(ctx context.Context) (*pdpb.Timestamp, error)

GetMaxLocalTSO will sync with the current Local TSO Allocators among the cluster to get the max Local TSO.

func (*AllocatorManager) GetSuffixBits

func (am *AllocatorManager) GetSuffixBits() int

GetSuffixBits calculates the bits of suffix sign by the max number of suffix so far, which will be used in the TSO logical part.

func (*AllocatorManager) HandleRequest

func (am *AllocatorManager) HandleRequest(dcLocation string, count uint32) (pdpb.Timestamp, error)

HandleRequest forwards TSO allocation requests to correct TSO Allocators.

func (*AllocatorManager) PriorityChecker

func (am *AllocatorManager) PriorityChecker()

PriorityChecker is used to check the election priority of a Local TSO Allocator. In the normal case, if we want to elect a Local TSO Allocator for a certain DC, such as dc-1, we need to make sure the follow priority rules: 1. The PD server with dc-location="dc-1" needs to be elected as the allocator leader with the highest priority. 2. If all PD servers with dc-location="dc-1" are down, then the other PD servers of DC could be elected.

func (*AllocatorManager) ResetAllocatorGroup

func (am *AllocatorManager) ResetAllocatorGroup(dcLocation string)

ResetAllocatorGroup will reset the allocator's leadership and TSO initialized in memory. It usually should be called before re-triggering an Allocator leader campaign.

func (*AllocatorManager) SetLocalTSOConfig

func (am *AllocatorManager) SetLocalTSOConfig(dcLocation string) error

SetLocalTSOConfig receives the zone label of this PD server and write it into etcd as dc-location to make the whole cluster know the DC-level topology for later Local TSO Allocator campaign.

func (*AllocatorManager) SetUpGlobalAllocator

func (am *AllocatorManager) SetUpGlobalAllocator(ctx context.Context, leadership *election.Leadership, startGlobalLeaderLoop bool)

SetUpGlobalAllocator is used to set up the global allocator, which will initialize the allocator and put it into an allocator daemon. An TSO Allocator should only be set once, and may be initialized and reset multiple times depending on the election.

func (*AllocatorManager) TransferAllocatorForDCLocation

func (am *AllocatorManager) TransferAllocatorForDCLocation(dcLocation string, memberID uint64) error

TransferAllocatorForDCLocation transfer local tso allocator to the target member for the given dcLocation

type Config

type Config interface {
	// GetLeaderLease returns the leader lease.
	GetLeaderLease() int64
	// IsLocalTSOEnabled returns if the local TSO is enabled.
	IsLocalTSOEnabled() bool
	// GetTSOUpdatePhysicalInterval returns TSO update physical interval.
	GetTSOUpdatePhysicalInterval() time.Duration
	// GetTSOSaveInterval returns TSO save interval.
	GetTSOSaveInterval() time.Duration
	// GetMaxResetTSGap returns the MaxResetTSGap.
	GetMaxResetTSGap() time.Duration
	// GetTLSConfig returns the TLS config.
	GetTLSConfig() *grpcutil.TLSConfig
}

Config is used to provide TSO configuration.

type DCLocationInfo

type DCLocationInfo struct {
	// dc-location/global (string) -> Member IDs
	ServerIDs []uint64
	// dc-location (string) -> Suffix sign. It is collected and maintained by the PD leader.
	Suffix int32
}

DCLocationInfo is used to record some dc-location related info, such as suffix sign and server IDs in this dc-location.

type ElectionMember

type ElectionMember interface {
	// ID returns the unique ID in the election group. For example, it can be unique
	// server id of a cluster or the unique keyspace group replica id of the election
	// group comprised of the replicas of a keyspace group.
	ID() uint64
	// ID returns the unique Name in the election group.
	Name() string
	// MemberValue returns the member value.
	MemberValue() string
	// GetMember() returns the current member
	GetMember() interface{}
	// Client returns the etcd client.
	Client() *clientv3.Client
	// IsLeader returns whether the participant is the leader or not by checking its
	// leadership's lease and leader info.
	IsLeader() bool
	// IsLeaderElected returns true if the leader exists; otherwise false.
	IsLeaderElected() bool
	// CheckLeader checks if someone else is taking the leadership. If yes, returns the leader;
	// otherwise returns a bool which indicates if it is needed to check later.
	CheckLeader() (leader member.ElectionLeader, checkAgain bool)
	// EnableLeader declares the member itself to be the leader.
	EnableLeader()
	// KeepLeader is used to keep the leader's leadership.
	KeepLeader(ctx context.Context)
	// CampaignLeader is used to campaign the leadership and make it become a leader in an election group.
	CampaignLeader(leaseTimeout int64) error
	// ResetLeader is used to reset the member's current leadership.
	// Basically it will reset the leader lease and unset leader info.
	ResetLeader()
	// GetLeaderListenUrls returns current leader's listen urls
	GetLeaderListenUrls() []string
	// GetLeaderID returns current leader's member ID.
	GetLeaderID() uint64
	// GetLeaderPath returns the path of the leader.
	GetLeaderPath() string
	// GetLeadership returns the leadership of the PD member.
	GetLeadership() *election.Leadership
	// GetLastLeaderUpdatedTime returns the last time when the leader is updated.
	GetLastLeaderUpdatedTime() time.Time
	// GetDCLocationPathPrefix returns the dc-location path prefix of the cluster.
	GetDCLocationPathPrefix() string
	// GetDCLocationPath returns the dc-location path of a member with the given member ID.
	GetDCLocationPath(id uint64) string
	// PreCheckLeader does some pre-check before checking whether it's the leader.
	PreCheckLeader() error
}

ElectionMember defines the interface for the election related logic.

type GlobalTSOAllocator

type GlobalTSOAllocator struct {
	// contains filtered or unexported fields
}

GlobalTSOAllocator is the global single point TSO allocator.

func (*GlobalTSOAllocator) GenerateTSO

func (gta *GlobalTSOAllocator) GenerateTSO(count uint32) (pdpb.Timestamp, error)

GenerateTSO is used to generate the given number of TSOs. Make sure you have initialized the TSO allocator before calling this method. Basically, there are two ways to generate a Global TSO:

  1. The old way to generate a normal TSO from memory directly, which makes the TSO service node become single point.
  2. The new way to generate a Global TSO by synchronizing with all other Local TSO Allocators.

And for the new way, there are two different strategies:

  1. Collect the max Local TSO from all Local TSO Allocator leaders and write it back to them as MaxTS.
  2. Estimate a MaxTS and try to write it to all Local TSO Allocator leaders directly to reduce the RTT. During the process, if the estimated MaxTS is not accurate, it will fallback to the collecting way.

func (*GlobalTSOAllocator) Initialize

func (gta *GlobalTSOAllocator) Initialize(int) error

Initialize will initialize the created global TSO allocator.

func (*GlobalTSOAllocator) IsInitialize

func (gta *GlobalTSOAllocator) IsInitialize() bool

IsInitialize is used to indicates whether this allocator is initialized.

func (*GlobalTSOAllocator) Reset

func (gta *GlobalTSOAllocator) Reset()

Reset is used to reset the TSO allocator.

func (*GlobalTSOAllocator) SetTSO

func (gta *GlobalTSOAllocator) SetTSO(tso uint64, ignoreSmaller, skipUpperBoundCheck bool) error

SetTSO sets the physical part with given TSO.

func (*GlobalTSOAllocator) SyncMaxTS

func (gta *GlobalTSOAllocator) SyncMaxTS(
	ctx context.Context,
	dcLocationMap map[string]DCLocationInfo,
	maxTSO *pdpb.Timestamp,
	skipCheck bool,
) error

SyncMaxTS is used to sync MaxTS with all Local TSO Allocator leaders in dcLocationMap. If maxTSO is the biggest TSO among all Local TSO Allocators, it will be written into each allocator and remains the same after the synchronization. If not, it will be replaced with the new max Local TSO and return.

func (*GlobalTSOAllocator) UpdateTSO

func (gta *GlobalTSOAllocator) UpdateTSO() error

UpdateTSO is used to update the TSO in memory and the time window in etcd.

type Handler

type Handler interface {
	ResetTS(ts uint64, ignoreSmaller, skipUpperBoundCheck bool, keyspaceGroupID uint32) error
}

Handler defines the common behaviors of a basic tso handler.

type KeyspaceGroupManager

type KeyspaceGroupManager struct {
	// contains filtered or unexported fields
}

KeyspaceGroupManager manages the members of the keyspace groups assigned to this host. The replicas campaign for the leaders which provide the tso service for the corresponding keyspace groups.

func NewKeyspaceGroupManager

func NewKeyspaceGroupManager(
	ctx context.Context,
	tsoServiceID *discovery.ServiceRegistryEntry,
	etcdClient *clientv3.Client,
	httpClient *http.Client,
	electionNamePrefix string,
	legacySvcRootPath string,
	tsoSvcRootPath string,
	cfg ServiceConfig,
) *KeyspaceGroupManager

NewKeyspaceGroupManager creates a new Keyspace Group Manager.

func (*KeyspaceGroupManager) Close

func (kgm *KeyspaceGroupManager) Close()

Close this KeyspaceGroupManager

func (*KeyspaceGroupManager) GetAllocatorManager

func (kgm *KeyspaceGroupManager) GetAllocatorManager(keyspaceGroupID uint32) (*AllocatorManager, error)

GetAllocatorManager returns the AllocatorManager of the given keyspace group

func (*KeyspaceGroupManager) GetElectionMember

func (kgm *KeyspaceGroupManager) GetElectionMember(
	keyspaceID, keyspaceGroupID uint32,
) (ElectionMember, error)

GetElectionMember returns the election member of the given keyspace group

func (*KeyspaceGroupManager) HandleTSORequest

func (kgm *KeyspaceGroupManager) HandleTSORequest(
	keyspaceID, keyspaceGroupID uint32,
	dcLocation string, count uint32,
) (ts pdpb.Timestamp, currentKeyspaceGroupID uint32, err error)

HandleTSORequest forwards TSO allocation requests to correct TSO Allocators of the given keyspace group.

func (*KeyspaceGroupManager) Initialize

func (kgm *KeyspaceGroupManager) Initialize() error

Initialize this KeyspaceGroupManager

type LocalTSOAllocator

type LocalTSOAllocator struct {
	// contains filtered or unexported fields
}

LocalTSOAllocator is the DC-level local TSO allocator, which is only used to allocate TSO in one DC each. One PD server may hold multiple Local TSO Allocators.

func (*LocalTSOAllocator) CampaignAllocatorLeader

func (lta *LocalTSOAllocator) CampaignAllocatorLeader(leaseTimeout int64, cmps ...clientv3.Cmp) error

CampaignAllocatorLeader is used to campaign a Local TSO Allocator's leadership.

func (*LocalTSOAllocator) CheckAllocatorLeader

func (lta *LocalTSOAllocator) CheckAllocatorLeader() (*pdpb.Member, int64, bool)

CheckAllocatorLeader checks who is the current Local TSO Allocator leader, and returns true if it is needed to check later.

func (*LocalTSOAllocator) EnableAllocatorLeader

func (lta *LocalTSOAllocator) EnableAllocatorLeader()

EnableAllocatorLeader sets the Local TSO Allocator itself to a leader.

func (*LocalTSOAllocator) GenerateTSO

func (lta *LocalTSOAllocator) GenerateTSO(count uint32) (pdpb.Timestamp, error)

GenerateTSO is used to generate a given number of TSOs. Make sure you have initialized the TSO allocator before calling.

func (*LocalTSOAllocator) GetAllocatorLeader

func (lta *LocalTSOAllocator) GetAllocatorLeader() *pdpb.Member

GetAllocatorLeader returns the Local TSO Allocator leader.

func (*LocalTSOAllocator) GetCurrentTSO

func (lta *LocalTSOAllocator) GetCurrentTSO() (*pdpb.Timestamp, error)

GetCurrentTSO returns current TSO in memory.

func (*LocalTSOAllocator) GetDCLocation

func (lta *LocalTSOAllocator) GetDCLocation() string

GetDCLocation returns the local allocator's dc-location.

func (*LocalTSOAllocator) GetMember

func (lta *LocalTSOAllocator) GetMember() ElectionMember

GetMember returns the Local TSO Allocator's member value.

func (*LocalTSOAllocator) Initialize

func (lta *LocalTSOAllocator) Initialize(suffix int) error

Initialize will initialize the created local TSO allocator.

func (*LocalTSOAllocator) IsAllocatorLeader

func (lta *LocalTSOAllocator) IsAllocatorLeader() bool

IsAllocatorLeader returns whether the allocator is still a Local TSO Allocator leader by checking its leadership's lease and leader info.

func (*LocalTSOAllocator) IsInitialize

func (lta *LocalTSOAllocator) IsInitialize() bool

IsInitialize is used to indicates whether this allocator is initialized.

func (*LocalTSOAllocator) KeepAllocatorLeader

func (lta *LocalTSOAllocator) KeepAllocatorLeader(ctx context.Context)

KeepAllocatorLeader is used to keep the PD leader's leadership.

func (*LocalTSOAllocator) Reset

func (lta *LocalTSOAllocator) Reset()

Reset is used to reset the TSO allocator.

func (*LocalTSOAllocator) SetTSO

func (lta *LocalTSOAllocator) SetTSO(tso uint64, ignoreSmaller, skipUpperBoundCheck bool) error

SetTSO sets the physical part with given TSO.

func (*LocalTSOAllocator) UpdateTSO

func (lta *LocalTSOAllocator) UpdateTSO() error

UpdateTSO is used to update the TSO in memory and the time window in etcd for all local TSO allocators this PD server hold.

func (*LocalTSOAllocator) WatchAllocatorLeader

func (lta *LocalTSOAllocator) WatchAllocatorLeader(serverCtx context.Context, allocatorLeader *pdpb.Member, revision int64)

WatchAllocatorLeader is used to watch the changes of the Local TSO Allocator leader.

func (*LocalTSOAllocator) WriteTSO

func (lta *LocalTSOAllocator) WriteTSO(maxTS *pdpb.Timestamp) error

WriteTSO is used to set the maxTS as current TSO in memory.

type ServiceConfig

type ServiceConfig interface {
	// GetName returns the Name
	GetName() string
	// GeBackendEndpoints returns the BackendEndpoints
	GeBackendEndpoints() string
	// GetListenAddr returns the ListenAddr
	GetListenAddr() string
	// GetAdvertiseListenAddr returns the AdvertiseListenAddr
	GetAdvertiseListenAddr() string
	// TSO-related configuration
	Config
}

ServiceConfig defines the configuration interface for the TSO service.

type TestServiceConfig

type TestServiceConfig struct {
	Name                      string              // Name of the service.
	BackendEndpoints          string              // Backend endpoints of the service.
	ListenAddr                string              // Address the service listens on.
	AdvertiseListenAddr       string              // Address the service advertises to the clients.
	LeaderLease               int64               // Leader lease.
	LocalTSOEnabled           bool                // Whether local TSO is enabled.
	TSOUpdatePhysicalInterval time.Duration       // Interval to update TSO in physical storage.
	TSOSaveInterval           time.Duration       // Interval to save TSO to physical storage.
	MaxResetTSGap             time.Duration       // Maximum gap to reset TSO.
	TLSConfig                 *grpcutil.TLSConfig // TLS configuration.
}

TestServiceConfig implements the ServiceConfig interface.

func (*TestServiceConfig) GeBackendEndpoints

func (c *TestServiceConfig) GeBackendEndpoints() string

GeBackendEndpoints returns the BackendEndpoints field of TestServiceConfig.

func (*TestServiceConfig) GetAdvertiseListenAddr

func (c *TestServiceConfig) GetAdvertiseListenAddr() string

GetAdvertiseListenAddr returns the AdvertiseListenAddr field of TestServiceConfig.

func (*TestServiceConfig) GetLeaderLease

func (c *TestServiceConfig) GetLeaderLease() int64

GetLeaderLease returns the LeaderLease field of TestServiceConfig.

func (*TestServiceConfig) GetListenAddr

func (c *TestServiceConfig) GetListenAddr() string

GetListenAddr returns the ListenAddr field of TestServiceConfig.

func (*TestServiceConfig) GetMaxResetTSGap

func (c *TestServiceConfig) GetMaxResetTSGap() time.Duration

GetMaxResetTSGap returns the MaxResetTSGap field of TestServiceConfig.

func (*TestServiceConfig) GetName

func (c *TestServiceConfig) GetName() string

GetName returns the Name field of TestServiceConfig.

func (*TestServiceConfig) GetTLSConfig

func (c *TestServiceConfig) GetTLSConfig() *grpcutil.TLSConfig

GetTLSConfig returns the TLSConfig field of TestServiceConfig.

func (*TestServiceConfig) GetTSOSaveInterval

func (c *TestServiceConfig) GetTSOSaveInterval() time.Duration

GetTSOSaveInterval returns the TSOSaveInterval field of TestServiceConfig.

func (*TestServiceConfig) GetTSOUpdatePhysicalInterval

func (c *TestServiceConfig) GetTSOUpdatePhysicalInterval() time.Duration

GetTSOUpdatePhysicalInterval returns the TSOUpdatePhysicalInterval field of TestServiceConfig.

func (*TestServiceConfig) IsLocalTSOEnabled

func (c *TestServiceConfig) IsLocalTSOEnabled() bool

IsLocalTSOEnabled returns the LocalTSOEnabled field of TestServiceConfig.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL