From e417bd204c87fcdf34252e220f36950c5cec517a Mon Sep 17 00:00:00 2001 From: Aylei Date: Mon, 9 Dec 2019 14:46:54 +0800 Subject: [PATCH 1/2] Add tikv config schema --- pkg/apis/pingcap/v1alpha1/tikv_config.go | 488 +++++++++++++++++++++++ 1 file changed, 488 insertions(+) create mode 100644 pkg/apis/pingcap/v1alpha1/tikv_config.go diff --git a/pkg/apis/pingcap/v1alpha1/tikv_config.go b/pkg/apis/pingcap/v1alpha1/tikv_config.go new file mode 100644 index 0000000000..a9b6df450b --- /dev/null +++ b/pkg/apis/pingcap/v1alpha1/tikv_config.go @@ -0,0 +1,488 @@ +// 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 v1alpha1 + +// Port from TiKV v3.0.6 + +// TiKVConfig is the configuration of TiKV. +// +k8s:openapi-gen=true +type TiKVConfig struct { + // +optional + LogLevel string `json:"log-level,omitempty",toml:"log-level,omitempty"` + // +optional + LogFile string `json:"log-file,omitempty",toml:"log-file,omitempty"` + // +optional + LogRotationTimespan string `json:"log-rotation-timespan,omitempty",toml:"log-rotation-timespan,omitempty"` + // +optional + PanicWhenUnexpectedKeyOrData *bool `json:"panic-when-unexpected-key-or-data,omitempty",toml:"panic-when-unexpected-key-or-data,omitempty"` + // +optional + Addr string `json:"addr,omitempty",toml:"addr,omitempty"` + // +optional + AdvertiseAddr string `json:"advertise-addr,omitempty",toml:"advertise-addr,omitempty"` + // +optional + StatusAddr string `json:"status-addr,omitempty",toml:"status-addr,omitempty"` + // +optional + StatusThreadPoolSize string `json:"status-thread-pool-size,omitempty",toml:"status-thread-pool-size,omitempty"` + // +optional + GrpcCompressionType string `json:"grpc-compression-type,omitempty",toml:"grpc-compression-type,omitempty"` + // +optional + GrpcConcurrency *uint `json:"grpc-concurrency,omitempty",toml:"grpc-concurrency,omitempty"` + // +optional + GrpcConcurrentStream *uint `json:"grpc-concurrent-stream,omitempty",toml:"grpc-concurrent-stream,omitempty"` + // +optional + GrpcRaftConnNum *uint `json:"grpc-raft-conn-num,omitempty",toml:"grpc-raft-conn-num,omitempty"` + // +optional + GrpcStreamInitialWindowSize string `json:"grpc-stream-initial-window-size,omitempty",toml:"grpc-stream-initial-window-size,omitempty"` + // +optional + GrpcKeepaliveTime string `json:"grpc-keepalive-time,omitempty",toml:"grpc-keepalive-time,omitempty"` + // +optional + GrpcKeepaliveTimeout string `json:"grpc-keepalive-timeout,omitempty",toml:"grpc-keepalive-timeout,omitempty"` + // +optional + ConcurrentSendSnapLimit *uint `json:"concurrent-send-snap-limit,omitempty",toml:"concurrent-send-snap-limit,omitempty"` + // +optional + ConcurrentRecvSnapLimit *uint `json:"concurrent-recv-snap-limit,omitempty",toml:"concurrent-recv-snap-limit,omitempty"` + // +optional + EndPointRecursionLimit *uint `json:"end-point-recursion-limit,omitempty",toml:"end-point-recursion-limit,omitempty"` + // +optional + EndPointStreamChannelSize *uint `json:"end-point-stream-channel-size,omitempty",toml:"end-point-stream-channel-size,omitempty"` + // +optional + EndPointBatchRowLimit *uint `json:"end-point-batch-row-limit,omitempty",toml:"end-point-batch-row-limit,omitempty"` + // +optional + EndPointStreamBatchRowLimit *uint `json:"end-point-stream-batch-row-limit,omitempty",toml:"end-point-stream-batch-row-limit,omitempty"` + // +optional + EndPointEnableBatchIfPossible *uint `json:"end-point-enable-batch-if-possible,omitempty",toml:"end-point-enable-batch-if-possible,omitempty"` + // +optional + EndPointRequestMaxHandleDuration string `json:"end-point-request-max-handle-duration,omitempty",toml:"end-point-request-max-handle-duration,omitempty"` + // +optional + SnapMaxWriteBytesPerSec string `json:"snap-max-write-bytes-per-sec,omitempty",toml:"snap-max-write-bytes-per-sec,omitempty"` + // +optional + SnapMaxTotalSize string `json:"snap-max-total-size,omitempty",toml:"snap-max-total-size,omitempty"` + // +optional + StatsConcurrency *uint `json:"stats-concurrency,omitempty",toml:"stats-concurrency,omitempty"` + // +optional + HeavyLoadThreshold *uint `json:"heavy-load-threshold,omitempty",toml:"heavy-load-threshold,omitempty"` + // +optional + HeavyLoadWaitDuration string `json:"heavy-load-wait-duration,omitempty",toml:"heavy-load-wait-duration,omitempty"` + // +optional + Labels map[string]string `json:"labels,omitempty",toml:"labels,omitempty"` + // +optional + Server *TiKVServerConfig `json:"server,omitempty",toml:"server,omitempty"` + // +optional + Storage *TiKVStorageConfig `json:"storage,omitempty",toml:"storage,omitempty"` + // +optional + Raftstore *TiKVRaftstoreConfig `json:"raftstore,omitempty",toml:"raftstore,omitempty"` + // +optional + Rocksdb *TiKVDbConfig `json:"rocksdb,omitempty",toml:"rocksdb,omitempty"` +} + +// TiKVDbConfig is the rocksdb config. +// +k8s:openapi-gen=true +type TiKVDbConfig struct { + // +optional + WalRecoveryMode *int64 `json:"wal-recovery-mode,omitempty",toml:"wal-recovery-mode,omitempty"` + // +optional + WalDir string `json:"wal-dir,omitempty",toml:"wal-dir,omitempty"` + // +optional + WalTTLSeconds *int64 `json:"wal-ttl-seconds,omitempty",toml:"wal-ttl-seconds,omitempty"` + // +optional + WalSizeLimit string `json:"wal-size-limit,omitempty",toml:"wal-size-limit,omitempty"` + // +optional + MaxTotalWalSize string `json:"max-total-wal-size,omitempty",toml:"max-total-wal-size,omitempty"` + // +optional + MaxBackgroundJobs *int64 `json:"max-background-jobs,omitempty",toml:"max-background-jobs,omitempty"` + // +optional + MaxManifestFileSize string `json:"max-manifest-file-size,omitempty",toml:"max-manifest-file-size,omitempty"` + // +optional + CreateIfMissing *bool `json:"create-if-missing,omitempty",toml:"create-if-missing,omitempty"` + // +optional + MaxOpenFiles *int64 `json:"max-open-files,omitempty",toml:"max-open-files,omitempty"` + // +optional + EnableStatistics *bool `json:"enable-statistics,omitempty",toml:"enable-statistics,omitempty"` + // +optional + StatsDumpPeriod string `json:"stats-dump-period,omitempty",toml:"stats-dump-period,omitempty"` + // +optional + CompactionReadaheadSize string `json:"compaction-readahead-size,omitempty",toml:"compaction-readahead-size,omitempty"` + // +optional + InfoLogMaxSize string `json:"info-log-max-size,omitempty",toml:"info-log-max-size,omitempty"` + // +optional + InfoLogRollTime string `json:"info-log-roll-time,omitempty",toml:"info-log-roll-time,omitempty"` + // +optional + InfoLogKeepLogFileNum *int64 `json:"info-log-keep-log-file-num,omitempty",toml:"info-log-keep-log-file-num,omitempty"` + // +optional + InfoLogDir string `json:"info-log-dir,omitempty",toml:"info-log-dir,omitempty"` + // +optional + RateBytesPerSec string `json:"rate-bytes-per-sec,omitempty",toml:"rate-bytes-per-sec,omitempty"` + // +optional + RateLimiterMode *int64 `json:"rate-limiter-mode,omitempty",toml:"rate-limiter-mode,omitempty"` + // +optional + AutoTuned *bool `json:"auto-tuned,omitempty",toml:"auto-tuned,omitempty"` + // +optional + BytesPerSync string `json:"bytes-per-sync,omitempty",toml:"bytes-per-sync,omitempty"` + // +optional + WalBytesPerSync string `json:"wal-bytes-per-sync,omitempty",toml:"wal-bytes-per-sync,omitempty"` + // +optional + MaxSubCompactions *int64 `json:"max-sub-compactions,omitempty",toml:"max-sub-compactions,omitempty"` + // +optional + WritableFileMaxBufferSize string `json:"writable-file-max-buffer-size,omitempty",toml:"writable-file-max-buffer-size,omitempty"` + // +optional + UseDirectIoForFlushAndCompaction *bool `json:"use-direct-io-for-flush-and-compaction,omitempty",toml:"use-direct-io-for-flush-and-compaction,omitempty"` + // +optional + EnablePipelinedWrite *bool `json:"enable-pipelined-write,omitempty",toml:"enable-pipelined-write,omitempty"` + // +optional + Defaultcf *TiKVCfConfig `json:"defaultcf,omitempty",toml:"defaultcf,omitempty"` + // +optional + Writecf *TiKVCfConfig `json:"writecf,omitempty",toml:"writecf,omitempty"` + // +optional + Lockcf *TiKVCfConfig `json:"lockcf,omitempty",toml:"lockcf,omitempty"` + // +optional + Raftcf *TiKVCfConfig `json:"raftcf,omitempty",toml:"raftcf,omitempty"` + // +optional + Titan *TiKVTitanDBConfig `json:"titan,omitempty",toml:"titan,omitempty"` +} + +// TiKVCfConfig is the config of a cf +// +k8s:openapi-gen=true +type TiKVCfConfig struct { + // +optional + BlockSize string `json:"block-size,omitempty",toml:"block-size,omitempty"` + // +optional + BlockCacheSize string `json:"block-cache-size,omitempty",toml:"block-cache-size,omitempty"` + // +optional + DisableBlockCache *bool `json:"disable-block-cache,omitempty",toml:"disable-block-cache,omitempty"` + // +optional + CacheIndexAndFilterBlocks *bool `json:"cache-index-and-filter-blocks,omitempty",toml:"cache-index-and-filter-blocks,omitempty"` + PinL0FilterAndIndexBlocks *bool `toml:"pin-l0-filter-and-index-blocks"` + // +optional + UseBloomFilter *bool `json:"use-bloom-filter,omitempty",toml:"use-bloom-filter,omitempty"` + // +optional + OptimizeFiltersForHits *bool `json:"optimize-filters-for-hits,omitempty",toml:"optimize-filters-for-hits,omitempty"` + // +optional + WholeKeyFiltering *bool `json:"whole-key-filtering,omitempty",toml:"whole-key-filtering,omitempty"` + // +optional + BloomFilterBitsPerKey *int64 `json:"bloom-filter-bits-per-key,omitempty",toml:"bloom-filter-bits-per-key,omitempty"` + // +optional + BlockBasedBloomFilter *bool `json:"block-based-bloom-filter,omitempty",toml:"block-based-bloom-filter,omitempty"` + // +optional + ReadAmpBytesPerBit *int64 `json:"read-amp-bytes-per-bit,omitempty",toml:"read-amp-bytes-per-bit,omitempty"` + // +optional + CompressionPerLevel []string `json:"compression-per-level,omitempty",toml:"compression-per-level,omitempty"` + // +optional + WriteBufferSize string `json:"write-buffer-size,omitempty",toml:"write-buffer-size,omitempty"` + // +optional + MaxWriteBufferNumber *int64 `json:"max-write-buffer-number,omitempty",toml:"max-write-buffer-number,omitempty"` + // +optional + MinWriteBufferNumberToMerge *int64 `json:"min-write-buffer-number-to-merge,omitempty",toml:"min-write-buffer-number-to-merge,omitempty"` + // +optional + MaxBytesForLevelBase string `json:"max-bytes-for-level-base,omitempty",toml:"max-bytes-for-level-base,omitempty"` + // +optional + TargetFileSizeBase string `json:"target-file-size-base,omitempty",toml:"target-file-size-base,omitempty"` + Level0FileNumCompactionTrigger *int64 `toml:"level0-file-num-compaction-trigger"` + Level0SlowdownWritesTrigger *int64 `toml:"level0-slowdown-writes-trigger"` + Level0StopWritesTrigger *int64 `toml:"level0-stop-writes-trigger"` + // +optional + MaxCompactionBytes string `json:"max-compaction-bytes,omitempty",toml:"max-compaction-bytes,omitempty"` + // +optional + CompactionPri *int64 `json:"compaction-pri,omitempty",toml:"compaction-pri,omitempty"` + // +optional + DynamicLevelBytes *bool `json:"dynamic-level-bytes,omitempty",toml:"dynamic-level-bytes,omitempty"` + // +optional + NumLevels *int64 `json:"num-levels,omitempty",toml:"num-levels,omitempty"` + // +optional + MaxBytesForLevelMultiplier *int64 `json:"max-bytes-for-level-multiplier,omitempty",toml:"max-bytes-for-level-multiplier,omitempty"` + // +optional + CompactionStyle *int64 `json:"compaction-style,omitempty",toml:"compaction-style,omitempty"` + // +optional + DisableAutoCompactions *bool `json:"disable-auto-compactions,omitempty",toml:"disable-auto-compactions,omitempty"` + // +optional + SoftPendingCompactionBytesLimit string `json:"soft-pending-compaction-bytes-limit,omitempty",toml:"soft-pending-compaction-bytes-limit,omitempty"` + // +optional + HardPendingCompactionBytesLimit string `json:"hard-pending-compaction-bytes-limit,omitempty",toml:"hard-pending-compaction-bytes-limit,omitempty"` + // +optional + ForceConsistencyChecks *bool `json:"force-consistency-checks,omitempty",toml:"force-consistency-checks,omitempty"` + // +optional + PropSizeIndexDistance *int64 `json:"prop-size-index-distance,omitempty",toml:"prop-size-index-distance,omitempty"` + // +optional + PropKeysIndexDistance *int64 `json:"prop-keys-index-distance,omitempty",toml:"prop-keys-index-distance,omitempty"` + // +optional + EnableDoublySkiplist *bool `json:"enable-doubly-skiplist,omitempty",toml:"enable-doubly-skiplist,omitempty"` + // +optional + Titan *TiKVTitanCfConfig `json:"titan,omitempty",toml:"titan,omitempty"` +} + +// TiKVTitanCfConfig is the titian config. +type TiKVTitanCfConfig struct { + // +optional + MinBlobSize string `json:"min-blob-size,omitempty",toml:"min-blob-size,omitempty"` + // +optional + BlobFileCompression string `json:"blob-file-compression,omitempty",toml:"blob-file-compression,omitempty"` + // +optional + BlobCacheSize string `json:"blob-cache-size,omitempty",toml:"blob-cache-size,omitempty"` + // +optional + MinGcBatchSize string `json:"min-gc-batch-size,omitempty",toml:"min-gc-batch-size,omitempty"` + // +optional + MaxGcBatchSize string `json:"max-gc-batch-size,omitempty",toml:"max-gc-batch-size,omitempty"` + // +optional + DiscardableRatio float64 `json:"discardable-ratio,omitempty",toml:"discardable-ratio,omitempty"` + // +optional + SampleRatio float64 `json:"sample-ratio,omitempty",toml:"sample-ratio,omitempty"` + // +optional + MergeSmallFileThreshold string `json:"merge-small-file-threshold,omitempty",toml:"merge-small-file-threshold,omitempty"` + // +optional + BlobRunMode string `json:"blob-run-mode,omitempty",toml:"blob-run-mode,omitempty"` +} + +// TiKVTitanDBConfig is the config a titian db. +// +k8s:openapi-gen=true +type TiKVTitanDBConfig struct { + // +optional + Enabled *bool `json:"enabled,omitempty",toml:"enabled,omitempty"` + // +optional + Dirname string `json:"dirname,omitempty",toml:"dirname,omitempty"` + // +optional + DisableGc *bool `json:"disable-gc,omitempty",toml:"disable-gc,omitempty"` + // +optional + MaxBackgroundGc *int64 `json:"max-background-gc,omitempty",toml:"max-background-gc,omitempty"` + // The value of this field will be truncated to seconds. + // +optional + PurgeObsoleteFilesPeriod string `json:"purge-obsolete-files-period,omitempty",toml:"purge-obsolete-files-period,omitempty"` +} + +// TiKVStorageConfig is the config of storage +// +k8s:openapi-gen=true +type TiKVStorageConfig struct { + // +optional + DataDir string `json:"data-dir,omitempty",toml:"data-dir,omitempty"` + // +optional + MaxKeySize *int64 `json:"max-key-size,omitempty",toml:"max-key-size,omitempty"` + // +optional + SchedulerNotifyCapacity *int64 `json:"scheduler-notify-capacity,omitempty",toml:"scheduler-notify-capacity,omitempty"` + // +optional + SchedulerConcurrency *int64 `json:"scheduler-concurrency,omitempty",toml:"scheduler-concurrency,omitempty"` + // +optional + SchedulerWorkerPoolSize *int64 `json:"scheduler-worker-pool-size,omitempty",toml:"scheduler-worker-pool-size,omitempty"` + // +optional + SchedulerPendingWriteThreshold string `json:"scheduler-pending-write-threshold,omitempty",toml:"scheduler-pending-write-threshold,omitempty"` + // +optional + BlockCache *TiKVBlockCacheConfig `json:"block-cache,omitempty",toml:"block-cache,omitempty"` +} + +// TiKVBlockCacheConfig is the config of a block cache +// +k8s:openapi-gen=true +type TiKVBlockCacheConfig struct { + // +optional + Shared *bool `json:"shared,omitempty",toml:"shared,omitempty"` + // +optional + Capacity string `json:"capacity,omitempty",toml:"capacity,omitempty"` + // +optional + NumShardBits *int64 `json:"num-shard-bits,omitempty",toml:"num-shard-bits,omitempty"` + // +optional + StrictCapacityLimit *bool `json:"strict-capacity-limit,omitempty",toml:"strict-capacity-limit,omitempty"` + // +optional + HighPriPoolRatio float64 `json:"high-pri-pool-ratio,omitempty",toml:"high-pri-pool-ratio,omitempty"` + // +optional + MemoryAllocator string `json:"memory-allocator,omitempty",toml:"memory-allocator,omitempty"` +} + +// TiKVServerConfig is the configuration of TiKV server. +// +k8s:openapi-gen=true +type TiKVServerConfig struct { + // +optional + GrpcCompressionType string `json:"grpc-compression-type,omitempty",toml:"grpc-compression-type,omitempty"` + // +optional + GrpcConcurrency *int64 `json:"grpc-concurrency,omitempty",toml:"grpc-concurrency,omitempty"` + // +optional + GrpcConcurrentStream *int64 `json:"grpc-concurrent-stream,omitempty",toml:"grpc-concurrent-stream,omitempty"` + // +optional + GrpcRaftConnNum *int64 `json:"grpc-raft-conn-num,omitempty",toml:"grpc-raft-conn-num,omitempty"` + // +optional + GrpcStreamInitialWindowSize string `json:"grpc-stream-initial-window-size,omitempty",toml:"grpc-stream-initial-window-size,omitempty"` + // +optional + GrpcKeepaliveTime string `json:"grpc-keepalive-time,omitempty",toml:"grpc-keepalive-time,omitempty"` + // +optional + GrpcKeepaliveTimeout string `json:"grpc-keepalive-timeout,omitempty",toml:"grpc-keepalive-timeout,omitempty"` + // +optional + ConcurrentSendSnapLimit *int64 `json:"concurrent-send-snap-limit,omitempty",toml:"concurrent-send-snap-limit,omitempty"` + // +optional + ConcurrentRecvSnapLimit *int64 `json:"concurrent-recv-snap-limit,omitempty",toml:"concurrent-recv-snap-limit,omitempty"` + // +optional + EndPointRecursionLimit *int64 `json:"end-point-recursion-limit,omitempty",toml:"end-point-recursion-limit,omitempty"` + // +optional + EndPointStreamChannelSize *int64 `json:"end-point-stream-channel-size,omitempty",toml:"end-point-stream-channel-size,omitempty"` + // +optional + EndPointBatchRowLimit *int64 `json:"end-point-batch-row-limit,omitempty",toml:"end-point-batch-row-limit,omitempty"` + // +optional + EndPointStreamBatchRowLimit *int64 `json:"end-point-stream-batch-row-limit,omitempty",toml:"end-point-stream-batch-row-limit,omitempty"` + // +optional + EndPointEnableBatchIfPossible *bool `json:"end-point-enable-batch-if-possible,omitempty",toml:"end-point-enable-batch-if-possible,omitempty"` + // +optional + EndPointRequestMaxHandleDuration string `json:"end-point-request-max-handle-duration,omitempty",toml:"end-point-request-max-handle-duration,omitempty"` + // +optional + SnapMaxWriteBytesPerSec string `json:"snap-max-write-bytes-per-sec,omitempty",toml:"snap-max-write-bytes-per-sec,omitempty"` + // +optional + SnapMaxTotalSize string `json:"snap-max-total-size,omitempty",toml:"snap-max-total-size,omitempty"` + // +optional + StatsConcurrency *int64 `json:"stats-concurrency,omitempty",toml:"stats-concurrency,omitempty"` + // +optional + HeavyLoadThreshold *int64 `json:"heavy-load-threshold,omitempty",toml:"heavy-load-threshold,omitempty"` + // +optional + HeavyLoadWaitDuration string `json:"heavy-load-wait-duration,omitempty",toml:"heavy-load-wait-duration,omitempty"` + // +optional + Labels map[string]string `json:"labels,omitempty",toml:"labels,omitempty"` +} + +// TiKVRaftstoreConfig is the configuration of TiKV raftstore component. +// +k8s:openapi-gen=true +type TiKVRaftstoreConfig struct { + // true for high reliability, prevent data loss when power failure. + // +optional + SyncLog *bool `json:"sync-log,omitempty",toml:"sync-log,omitempty"` + + // raft-base-tick-interval is a base tick interval (ms). + // +optional + RaftBaseTickInterval string `json:"raft-base-tick-interval,omitempty",toml:"raft-base-tick-interval,omitempty"` + // +optional + RaftHeartbeatTicks *int64 `json:"raft-heartbeat-ticks,omitempty",toml:"raft-heartbeat-ticks,omitempty"` + // +optional + RaftElectionTimeoutTicks *int64 `json:"raft-election-timeout-ticks,omitempty",toml:"raft-election-timeout-ticks,omitempty"` + // When the entry exceed the max size, reject to propose it. + // +optional + RaftEntryMaxSize string `json:"raft-entry-max-size,omitempty",toml:"raft-entry-max-size,omitempty"` + + // Interval to gc unnecessary raft log (ms). + // +optional + RaftLogGCTickInterval string `json:"raft-log-gc-tick-interval,omitempty",toml:"raft-log-gc-tick-interval,omitempty"` + // A threshold to gc stale raft log, must >= 1. + // +optional + RaftLogGCThreshold *int64 `json:"raft-log-gc-threshold,omitempty",toml:"raft-log-gc-threshold,omitempty"` + // When entry count exceed this value, gc will be forced trigger. + // +optional + RaftLogGCCountLimit *int64 `json:"raft-log-gc-count-limit,omitempty",toml:"raft-log-gc-count-limit,omitempty"` + // When the approximate size of raft log entries exceed this value + // gc will be forced trigger. + // +optional + RaftLogGCSizeLimit string `json:"raft-log-gc-size-limit,omitempty",toml:"raft-log-gc-size-limit,omitempty"` + // When a peer is not responding for this time, leader will not keep entry cache for it. + // +optional + RaftEntryCacheLifeTime string `json:"raft-entry-cache-life-time,omitempty",toml:"raft-entry-cache-life-time,omitempty"` + // When a peer is newly added, reject transferring leader to the peer for a while. + // +optional + RaftRejectTransferLeaderDuration string `json:"raft-reject-transfer-leader-duration,omitempty",toml:"raft-reject-transfer-leader-duration,omitempty"` + + // Interval (ms) to check region whether need to be split or not. + // +optional + SplitRegionCheckTickInterval string `json:"split-region-check-tick-interval,omitempty",toml:"split-region-check-tick-interval,omitempty"` + /// When size change of region exceed the diff since last check, it + /// will be checked again whether it should be split. + // +optional + RegionSplitCheckDiff string `json:"region-split-check-diff,omitempty",toml:"region-split-check-diff,omitempty"` + /// Interval (ms) to check whether start compaction for a region. + // +optional + RegionCompactCheckInterval string `json:"region-compact-check-interval,omitempty",toml:"region-compact-check-interval,omitempty"` + // delay time before deleting a stale peer + // +optional + CleanStalePeerDelay string `json:"clean-stale-peer-delay,omitempty",toml:"clean-stale-peer-delay,omitempty"` + /// Number of regions for each time checking. + // +optional + RegionCompactCheckStep *int64 `json:"region-compact-check-step,omitempty",toml:"region-compact-check-step,omitempty"` + /// Minimum number of tombstones to trigger manual compaction. + // +optional + RegionCompactMinTombstones *int64 `json:"region-compact-min-tombstones,omitempty",toml:"region-compact-min-tombstones,omitempty"` + /// Minimum percentage of tombstones to trigger manual compaction. + /// Should between 1 and 100. + // +optional + RegionCompactTombstonesPercent *int64 `json:"region-compact-tombstones-percent,omitempty",toml:"region-compact-tombstones-percent,omitempty"` + // +optional + PdHeartbeatTickInterval string `json:"pd-heartbeat-tick-interval,omitempty",toml:"pd-heartbeat-tick-interval,omitempty"` + // +optional + PdStoreHeartbeatTickInterval string `json:"pd-store-heartbeat-tick-interval,omitempty",toml:"pd-store-heartbeat-tick-interval,omitempty"` + // +optional + SnapMgrGCTickInterval string `json:"snap-mgr-gc-tick-interval,omitempty",toml:"snap-mgr-gc-tick-interval,omitempty"` + // +optional + SnapGCTimeout string `json:"snap-gc-timeout,omitempty",toml:"snap-gc-timeout,omitempty"` + // +optional + LockCfCompactInterval string `json:"lock-cf-compact-interval,omitempty",toml:"lock-cf-compact-interval,omitempty"` + // +optional + LockCfCompactBytesThreshold string `json:"lock-cf-compact-bytes-threshold,omitempty",toml:"lock-cf-compact-bytes-threshold,omitempty"` + + // +optional + NotifyCapacity *int64 `json:"notify-capacity,omitempty",toml:"notify-capacity,omitempty"` + // +optional + MessagesPerTick *int64 `json:"messages-per-tick,omitempty",toml:"messages-per-tick,omitempty"` + + /// When a peer is not active for max-peer-down-duration + /// the peer is considered to be down and is reported to PD. + // +optional + MaxPeerDownDuration string `json:"max-peer-down-duration,omitempty",toml:"max-peer-down-duration,omitempty"` + + /// If the leader of a peer is missing for longer than max-leader-missing-duration + /// the peer would ask pd to confirm whether it is valid in any region. + /// If the peer is stale and is not valid in any region, it will destroy itself. + // +optional + MaxLeaderMissingDuration string `json:"max-leader-missing-duration,omitempty",toml:"max-leader-missing-duration,omitempty"` + /// Similar to the max-leader-missing-duration, instead it will log warnings and + /// try to alert monitoring systems, if there is any. + // +optional + AbnormalLeaderMissingDuration string `json:"abnormal-leader-missing-duration,omitempty",toml:"abnormal-leader-missing-duration,omitempty"` + // +optional + PeerStaleStateCheckInterval string `json:"peer-stale-state-check-interval,omitempty",toml:"peer-stale-state-check-interval,omitempty"` + + // +optional + LeaderTransferMaxLogLag *int64 `json:"leader-transfer-max-log-lag,omitempty",toml:"leader-transfer-max-log-lag,omitempty"` + + // +optional + SnapApplyBatchSize string `json:"snap-apply-batch-size,omitempty",toml:"snap-apply-batch-size,omitempty"` + + // Interval (ms) to check region whether the data is consistent. + // +optional + ConsistencyCheckInterval string `json:"consistency-check-interval,omitempty",toml:"consistency-check-interval,omitempty"` + + // +optional + ReportRegionFlowInterval string `json:"report-region-flow-interval,omitempty",toml:"report-region-flow-interval,omitempty"` + + // The lease provided by a successfully proposed and applied entry. + // +optional + RaftStoreMaxLeaderLease string `json:"raft-store-max-leader-lease,omitempty",toml:"raft-store-max-leader-lease,omitempty"` + + // Right region derive origin region id when split. + // +optional + RightDeriveWhenSplit *bool `json:"right-derive-when-split,omitempty",toml:"right-derive-when-split,omitempty"` + + // +optional + AllowRemoveLeader *bool `json:"allow-remove-leader,omitempty",toml:"allow-remove-leader,omitempty"` + + /// Max log gap allowed to propose merge. + // +optional + MergeMaxLogGap *int64 `json:"merge-max-log-gap,omitempty",toml:"merge-max-log-gap,omitempty"` + /// Interval to re-propose merge. + // +optional + MergeCheckTickInterval string `json:"merge-check-tick-interval,omitempty",toml:"merge-check-tick-interval,omitempty"` + + // +optional + UseDeleteRange *bool `json:"use-delete-range,omitempty",toml:"use-delete-range,omitempty"` + + // +optional + CleanupImportSstInterval string `json:"cleanup-import-sst-interval,omitempty",toml:"cleanup-import-sst-interval,omitempty"` + + // +optional + ApplyMaxBatchSize *int64 `json:"apply-max-batch-size,omitempty",toml:"apply-max-batch-size,omitempty"` + // +optional + ApplyPoolSize *int64 `json:"apply-pool-size,omitempty",toml:"apply-pool-size,omitempty"` + + // +optional + StoreMaxBatchSize *int64 `json:"store-max-batch-size,omitempty",toml:"store-max-batch-size,omitempty"` + // +optional + StorePoolSize *int64 `json:"store-pool-size,omitempty",toml:"store-pool-size,omitempty"` + // +optional + HibernateRegions *bool `json:"hibernate-regions,omitempty",toml:"hibernate-regions,omitempty"` +} From c7787c86db12f1a971e3b3f60b7fd9f8edb92436 Mon Sep 17 00:00:00 2001 From: Aylei Date: Mon, 9 Dec 2019 17:43:36 +0800 Subject: [PATCH 2/2] Add schema for TiKV Signed-off-by: Aylei --- manifests/crd.yaml | 771 ++++++++++ .../pingcap/v1alpha1/openapi_generated.go | 1280 ++++++++++++++++- pkg/apis/pingcap/v1alpha1/tikv_config.go | 374 ++--- pkg/apis/pingcap/v1alpha1/types.go | 7 +- .../pingcap/v1alpha1/zz_generated.deepcopy.go | 679 ++++++++- 5 files changed, 2908 insertions(+), 203 deletions(-) diff --git a/manifests/crd.yaml b/manifests/crd.yaml index 8791ab70c6..e710a33749 100644 --- a/manifests/crd.yaml +++ b/manifests/crd.yaml @@ -1421,6 +1421,777 @@ spec: tikv: description: TiKVSpec contains details of TiKV members properties: + config: + description: TiKVConfig is the configuration of TiKV. + properties: + addr: + type: string + advertise-addr: + type: string + concurrent-recv-snap-limit: + format: int32 + type: integer + concurrent-send-snap-limit: + format: int32 + type: integer + end-point-batch-row-limit: + format: int32 + type: integer + end-point-enable-batch-if-possible: + format: int32 + type: integer + end-point-recursion-limit: + format: int32 + type: integer + end-point-request-max-handle-duration: + type: string + end-point-stream-batch-row-limit: + format: int32 + type: integer + end-point-stream-channel-size: + format: int32 + type: integer + grpc-compression-type: + type: string + grpc-concurrency: + format: int32 + type: integer + grpc-concurrent-stream: + format: int32 + type: integer + grpc-keepalive-time: + type: string + grpc-keepalive-timeout: + type: string + grpc-raft-conn-num: + format: int32 + type: integer + grpc-stream-initial-window-size: + type: string + heavy-load-threshold: + format: int32 + type: integer + heavy-load-wait-duration: + type: string + labels: + type: object + log-file: + type: string + log-level: + type: string + log-rotation-timespan: + type: string + panic-when-unexpected-key-or-data: + type: boolean + raftstore: + description: TiKVRaftstoreConfig is the configuration of TiKV + raftstore component. + properties: + abnormal-leader-missing-duration: + description: / Similar to the max-leader-missing-duration, + instead it will log warnings and / try to alert monitoring + systems, if there is any. + type: string + allow-remove-leader: + type: boolean + apply-max-batch-size: + format: int64 + type: integer + apply-pool-size: + format: int64 + type: integer + clean-stale-peer-delay: + description: delay time before deleting a stale peer + type: string + cleanup-import-sst-interval: + type: string + consistency-check-interval: + description: Interval (ms) to check region whether the data + is consistent. + type: string + hibernate-regions: + type: boolean + leader-transfer-max-log-lag: + format: int64 + type: integer + lock-cf-compact-bytes-threshold: + type: string + lock-cf-compact-interval: + type: string + max-leader-missing-duration: + description: / If the leader of a peer is missing for longer + than max-leader-missing-duration / the peer would ask + pd to confirm whether it is valid in any region. / If + the peer is stale and is not valid in any region, it will + destroy itself. + type: string + max-peer-down-duration: + description: / When a peer is not active for max-peer-down-duration + / the peer is considered to be down and is reported to + PD. + type: string + merge-check-tick-interval: + description: / Interval to re-propose merge. + type: string + merge-max-log-gap: + description: / Max log gap allowed to propose merge. + format: int64 + type: integer + messages-per-tick: + format: int64 + type: integer + notify-capacity: + format: int64 + type: integer + pd-heartbeat-tick-interval: + type: string + pd-store-heartbeat-tick-interval: + type: string + peer-stale-state-check-interval: + type: string + raft-base-tick-interval: + description: raft-base-tick-interval is a base tick interval + (ms). + type: string + raft-election-timeout-ticks: + format: int64 + type: integer + raft-entry-cache-life-time: + description: When a peer is not responding for this time, + leader will not keep entry cache for it. + type: string + raft-entry-max-size: + description: When the entry exceed the max size, reject + to propose it. + type: string + raft-heartbeat-ticks: + format: int64 + type: integer + raft-log-gc-count-limit: + description: When entry count exceed this value, gc will + be forced trigger. + format: int64 + type: integer + raft-log-gc-size-limit: + description: When the approximate size of raft log entries + exceed this value gc will be forced trigger. + type: string + raft-log-gc-threshold: + description: A threshold to gc stale raft log, must >= 1. + format: int64 + type: integer + raft-log-gc-tick-interval: + description: Interval to gc unnecessary raft log (ms). + type: string + raft-reject-transfer-leader-duration: + description: When a peer is newly added, reject transferring + leader to the peer for a while. + type: string + raft-store-max-leader-lease: + description: The lease provided by a successfully proposed + and applied entry. + type: string + region-compact-check-interval: + description: / Interval (ms) to check whether start compaction + for a region. + type: string + region-compact-check-step: + description: / Number of regions for each time checking. + format: int64 + type: integer + region-compact-min-tombstones: + description: / Minimum number of tombstones to trigger manual + compaction. + format: int64 + type: integer + region-compact-tombstones-percent: + description: / Minimum percentage of tombstones to trigger + manual compaction. / Should between 1 and 100. + format: int64 + type: integer + region-split-check-diff: + description: / When size change of region exceed the diff + since last check, it / will be checked again whether it + should be split. + type: string + report-region-flow-interval: + type: string + right-derive-when-split: + description: Right region derive origin region id when split. + type: boolean + snap-apply-batch-size: + type: string + snap-gc-timeout: + type: string + snap-mgr-gc-tick-interval: + type: string + split-region-check-tick-interval: + description: Interval (ms) to check region whether need + to be split or not. + type: string + store-max-batch-size: + format: int64 + type: integer + store-pool-size: + format: int64 + type: integer + sync-log: + description: true for high reliability, prevent data loss + when power failure. + type: boolean + use-delete-range: + type: boolean + type: object + rocksdb: + description: TiKVDbConfig is the rocksdb config. + properties: + auto-tuned: + type: boolean + bytes-per-sync: + type: string + compaction-readahead-size: + type: string + create-if-missing: + type: boolean + defaultcf: + description: TiKVCfConfig is the config of a cf + properties: + Level0FileNumCompactionTrigger: + format: int64 + type: integer + Level0SlowdownWritesTrigger: + format: int64 + type: integer + Level0StopWritesTrigger: + format: int64 + type: integer + PinL0FilterAndIndexBlocks: + type: boolean + block-based-bloom-filter: + type: boolean + block-cache-size: + type: string + block-size: + type: string + bloom-filter-bits-per-key: + format: int64 + type: integer + cache-index-and-filter-blocks: + type: boolean + compaction-pri: + format: int64 + type: integer + compaction-style: + format: int64 + type: integer + compression-per-level: + items: + type: string + type: array + disable-auto-compactions: + type: boolean + disable-block-cache: + type: boolean + dynamic-level-bytes: + type: boolean + enable-doubly-skiplist: + type: boolean + force-consistency-checks: + type: boolean + hard-pending-compaction-bytes-limit: + type: string + max-bytes-for-level-base: + type: string + max-bytes-for-level-multiplier: + format: int64 + type: integer + max-compaction-bytes: + type: string + max-write-buffer-number: + format: int64 + type: integer + min-write-buffer-number-to-merge: + format: int64 + type: integer + num-levels: + format: int64 + type: integer + optimize-filters-for-hits: + type: boolean + prop-keys-index-distance: + format: int64 + type: integer + prop-size-index-distance: + format: int64 + type: integer + read-amp-bytes-per-bit: + format: int64 + type: integer + soft-pending-compaction-bytes-limit: + type: string + target-file-size-base: + type: string + titan: {} + use-bloom-filter: + type: boolean + whole-key-filtering: + type: boolean + write-buffer-size: + type: string + required: + - PinL0FilterAndIndexBlocks + - Level0FileNumCompactionTrigger + - Level0SlowdownWritesTrigger + - Level0StopWritesTrigger + type: object + enable-pipelined-write: + type: boolean + enable-statistics: + type: boolean + info-log-dir: + type: string + info-log-keep-log-file-num: + format: int64 + type: integer + info-log-max-size: + type: string + info-log-roll-time: + type: string + lockcf: + description: TiKVCfConfig is the config of a cf + properties: + Level0FileNumCompactionTrigger: + format: int64 + type: integer + Level0SlowdownWritesTrigger: + format: int64 + type: integer + Level0StopWritesTrigger: + format: int64 + type: integer + PinL0FilterAndIndexBlocks: + type: boolean + block-based-bloom-filter: + type: boolean + block-cache-size: + type: string + block-size: + type: string + bloom-filter-bits-per-key: + format: int64 + type: integer + cache-index-and-filter-blocks: + type: boolean + compaction-pri: + format: int64 + type: integer + compaction-style: + format: int64 + type: integer + compression-per-level: + items: + type: string + type: array + disable-auto-compactions: + type: boolean + disable-block-cache: + type: boolean + dynamic-level-bytes: + type: boolean + enable-doubly-skiplist: + type: boolean + force-consistency-checks: + type: boolean + hard-pending-compaction-bytes-limit: + type: string + max-bytes-for-level-base: + type: string + max-bytes-for-level-multiplier: + format: int64 + type: integer + max-compaction-bytes: + type: string + max-write-buffer-number: + format: int64 + type: integer + min-write-buffer-number-to-merge: + format: int64 + type: integer + num-levels: + format: int64 + type: integer + optimize-filters-for-hits: + type: boolean + prop-keys-index-distance: + format: int64 + type: integer + prop-size-index-distance: + format: int64 + type: integer + read-amp-bytes-per-bit: + format: int64 + type: integer + soft-pending-compaction-bytes-limit: + type: string + target-file-size-base: + type: string + titan: {} + use-bloom-filter: + type: boolean + whole-key-filtering: + type: boolean + write-buffer-size: + type: string + required: + - PinL0FilterAndIndexBlocks + - Level0FileNumCompactionTrigger + - Level0SlowdownWritesTrigger + - Level0StopWritesTrigger + type: object + max-background-jobs: + format: int64 + type: integer + max-manifest-file-size: + type: string + max-open-files: + format: int64 + type: integer + max-sub-compactions: + format: int64 + type: integer + max-total-wal-size: + type: string + raftcf: + description: TiKVCfConfig is the config of a cf + properties: + Level0FileNumCompactionTrigger: + format: int64 + type: integer + Level0SlowdownWritesTrigger: + format: int64 + type: integer + Level0StopWritesTrigger: + format: int64 + type: integer + PinL0FilterAndIndexBlocks: + type: boolean + block-based-bloom-filter: + type: boolean + block-cache-size: + type: string + block-size: + type: string + bloom-filter-bits-per-key: + format: int64 + type: integer + cache-index-and-filter-blocks: + type: boolean + compaction-pri: + format: int64 + type: integer + compaction-style: + format: int64 + type: integer + compression-per-level: + items: + type: string + type: array + disable-auto-compactions: + type: boolean + disable-block-cache: + type: boolean + dynamic-level-bytes: + type: boolean + enable-doubly-skiplist: + type: boolean + force-consistency-checks: + type: boolean + hard-pending-compaction-bytes-limit: + type: string + max-bytes-for-level-base: + type: string + max-bytes-for-level-multiplier: + format: int64 + type: integer + max-compaction-bytes: + type: string + max-write-buffer-number: + format: int64 + type: integer + min-write-buffer-number-to-merge: + format: int64 + type: integer + num-levels: + format: int64 + type: integer + optimize-filters-for-hits: + type: boolean + prop-keys-index-distance: + format: int64 + type: integer + prop-size-index-distance: + format: int64 + type: integer + read-amp-bytes-per-bit: + format: int64 + type: integer + soft-pending-compaction-bytes-limit: + type: string + target-file-size-base: + type: string + titan: {} + use-bloom-filter: + type: boolean + whole-key-filtering: + type: boolean + write-buffer-size: + type: string + required: + - PinL0FilterAndIndexBlocks + - Level0FileNumCompactionTrigger + - Level0SlowdownWritesTrigger + - Level0StopWritesTrigger + type: object + rate-bytes-per-sec: + type: string + rate-limiter-mode: + format: int64 + type: integer + stats-dump-period: + type: string + titan: + description: TiKVTitanDBConfig is the config a titian db. + properties: + dirname: + type: string + disable-gc: + type: boolean + enabled: + type: boolean + max-background-gc: + format: int64 + type: integer + purge-obsolete-files-period: + description: The value of this field will be truncated + to seconds. + type: string + type: object + use-direct-io-for-flush-and-compaction: + type: boolean + wal-bytes-per-sync: + type: string + wal-dir: + type: string + wal-recovery-mode: + format: int64 + type: integer + wal-size-limit: + type: string + wal-ttl-seconds: + format: int64 + type: integer + writable-file-max-buffer-size: + type: string + writecf: + description: TiKVCfConfig is the config of a cf + properties: + Level0FileNumCompactionTrigger: + format: int64 + type: integer + Level0SlowdownWritesTrigger: + format: int64 + type: integer + Level0StopWritesTrigger: + format: int64 + type: integer + PinL0FilterAndIndexBlocks: + type: boolean + block-based-bloom-filter: + type: boolean + block-cache-size: + type: string + block-size: + type: string + bloom-filter-bits-per-key: + format: int64 + type: integer + cache-index-and-filter-blocks: + type: boolean + compaction-pri: + format: int64 + type: integer + compaction-style: + format: int64 + type: integer + compression-per-level: + items: + type: string + type: array + disable-auto-compactions: + type: boolean + disable-block-cache: + type: boolean + dynamic-level-bytes: + type: boolean + enable-doubly-skiplist: + type: boolean + force-consistency-checks: + type: boolean + hard-pending-compaction-bytes-limit: + type: string + max-bytes-for-level-base: + type: string + max-bytes-for-level-multiplier: + format: int64 + type: integer + max-compaction-bytes: + type: string + max-write-buffer-number: + format: int64 + type: integer + min-write-buffer-number-to-merge: + format: int64 + type: integer + num-levels: + format: int64 + type: integer + optimize-filters-for-hits: + type: boolean + prop-keys-index-distance: + format: int64 + type: integer + prop-size-index-distance: + format: int64 + type: integer + read-amp-bytes-per-bit: + format: int64 + type: integer + soft-pending-compaction-bytes-limit: + type: string + target-file-size-base: + type: string + titan: {} + use-bloom-filter: + type: boolean + whole-key-filtering: + type: boolean + write-buffer-size: + type: string + required: + - PinL0FilterAndIndexBlocks + - Level0FileNumCompactionTrigger + - Level0SlowdownWritesTrigger + - Level0StopWritesTrigger + type: object + type: object + server: + description: TiKVServerConfig is the configuration of TiKV server. + properties: + concurrent-recv-snap-limit: + format: int64 + type: integer + concurrent-send-snap-limit: + format: int64 + type: integer + end-point-batch-row-limit: + format: int64 + type: integer + end-point-enable-batch-if-possible: + type: boolean + end-point-recursion-limit: + format: int64 + type: integer + end-point-request-max-handle-duration: + type: string + end-point-stream-batch-row-limit: + format: int64 + type: integer + end-point-stream-channel-size: + format: int64 + type: integer + grpc-compression-type: + type: string + grpc-concurrency: + format: int64 + type: integer + grpc-concurrent-stream: + format: int64 + type: integer + grpc-keepalive-time: + type: string + grpc-keepalive-timeout: + type: string + grpc-raft-conn-num: + format: int64 + type: integer + grpc-stream-initial-window-size: + type: string + heavy-load-threshold: + format: int64 + type: integer + heavy-load-wait-duration: + type: string + labels: + type: object + snap-max-total-size: + type: string + snap-max-write-bytes-per-sec: + type: string + stats-concurrency: + format: int64 + type: integer + type: object + snap-max-total-size: + type: string + snap-max-write-bytes-per-sec: + type: string + stats-concurrency: + format: int32 + type: integer + status-addr: + type: string + status-thread-pool-size: + type: string + storage: + description: TiKVStorageConfig is the config of storage + properties: + block-cache: + description: TiKVBlockCacheConfig is the config of a block + cache + properties: + capacity: + type: string + high-pri-pool-ratio: + format: double + type: number + memory-allocator: + type: string + num-shard-bits: + format: int64 + type: integer + shared: + type: boolean + strict-capacity-limit: + type: boolean + type: object + data-dir: + type: string + max-key-size: + format: int64 + type: integer + scheduler-concurrency: + format: int64 + type: integer + scheduler-notify-capacity: + format: int64 + type: integer + scheduler-pending-write-threshold: + type: string + scheduler-worker-pool-size: + format: int64 + type: integer + type: object + type: object maxFailoverCount: format: int32 type: integer diff --git a/pkg/apis/pingcap/v1alpha1/openapi_generated.go b/pkg/apis/pingcap/v1alpha1/openapi_generated.go index 1c89dce2bd..b3bf119082 100644 --- a/pkg/apis/pingcap/v1alpha1/openapi_generated.go +++ b/pkg/apis/pingcap/v1alpha1/openapi_generated.go @@ -74,8 +74,16 @@ func GetOpenAPIDefinitions(ref common.ReferenceCallback) map[string]common.OpenA "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiDBServiceSpec": schema_pkg_apis_pingcap_v1alpha1_TiDBServiceSpec(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiDBSlowLogTailerSpec": schema_pkg_apis_pingcap_v1alpha1_TiDBSlowLogTailerSpec(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiDBSpec": schema_pkg_apis_pingcap_v1alpha1_TiDBSpec(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVBlockCacheConfig": schema_pkg_apis_pingcap_v1alpha1_TiKVBlockCacheConfig(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVCfConfig": schema_pkg_apis_pingcap_v1alpha1_TiKVCfConfig(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVClient": schema_pkg_apis_pingcap_v1alpha1_TiKVClient(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVConfig": schema_pkg_apis_pingcap_v1alpha1_TiKVConfig(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVDbConfig": schema_pkg_apis_pingcap_v1alpha1_TiKVDbConfig(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVRaftstoreConfig": schema_pkg_apis_pingcap_v1alpha1_TiKVRaftstoreConfig(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVServerConfig": schema_pkg_apis_pingcap_v1alpha1_TiKVServerConfig(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVSpec": schema_pkg_apis_pingcap_v1alpha1_TiKVSpec(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVStorageConfig": schema_pkg_apis_pingcap_v1alpha1_TiKVStorageConfig(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVTitanDBConfig": schema_pkg_apis_pingcap_v1alpha1_TiKVTitanDBConfig(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TidbCluster": schema_pkg_apis_pingcap_v1alpha1_TidbCluster(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TidbClusterList": schema_pkg_apis_pingcap_v1alpha1_TidbClusterList(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TidbClusterSpec": schema_pkg_apis_pingcap_v1alpha1_TidbClusterSpec(ref), @@ -1798,7 +1806,7 @@ func schema_pkg_apis_pingcap_v1alpha1_PDSpec(ref common.ReferenceCallback) commo }, "config": { SchemaProps: spec.SchemaProps{ - Description: "Config is the Configuration of tidb-servers", + Description: "Config is the Configuration of pd-servers", Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.PDConfig"), }, }, @@ -2916,6 +2924,281 @@ func schema_pkg_apis_pingcap_v1alpha1_TiDBSpec(ref common.ReferenceCallback) com } } +func schema_pkg_apis_pingcap_v1alpha1_TiKVBlockCacheConfig(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "TiKVBlockCacheConfig is the config of a block cache", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "shared": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "capacity": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "num-shard-bits": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "strict-capacity-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "high-pri-pool-ratio": { + SchemaProps: spec.SchemaProps{ + Type: []string{"number"}, + Format: "double", + }, + }, + "memory-allocator": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + }, + }, + }, + } +} + +func schema_pkg_apis_pingcap_v1alpha1_TiKVCfConfig(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "TiKVCfConfig is the config of a cf", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "block-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "block-cache-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "disable-block-cache": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "cache-index-and-filter-blocks": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "PinL0FilterAndIndexBlocks": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "use-bloom-filter": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "optimize-filters-for-hits": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "whole-key-filtering": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "bloom-filter-bits-per-key": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "block-based-bloom-filter": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "read-amp-bytes-per-bit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "compression-per-level": { + SchemaProps: spec.SchemaProps{ + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + }, + }, + }, + "write-buffer-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "max-write-buffer-number": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "min-write-buffer-number-to-merge": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "max-bytes-for-level-base": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "target-file-size-base": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "Level0FileNumCompactionTrigger": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "Level0SlowdownWritesTrigger": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "Level0StopWritesTrigger": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "max-compaction-bytes": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "compaction-pri": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "dynamic-level-bytes": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "num-levels": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "max-bytes-for-level-multiplier": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "compaction-style": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "disable-auto-compactions": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "soft-pending-compaction-bytes-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "hard-pending-compaction-bytes-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "force-consistency-checks": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "prop-size-index-distance": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "prop-keys-index-distance": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "enable-doubly-skiplist": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "titan": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVTitanCfConfig"), + }, + }, + }, + Required: []string{"PinL0FilterAndIndexBlocks", "Level0FileNumCompactionTrigger", "Level0SlowdownWritesTrigger", "Level0StopWritesTrigger"}, + }, + }, + Dependencies: []string{ + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVTitanCfConfig"}, + } +} + func schema_pkg_apis_pingcap_v1alpha1_TiKVClient(ref common.ReferenceCallback) common.OpenAPIDefinition { return common.OpenAPIDefinition{ Schema: spec.Schema{ @@ -3006,41 +3289,1016 @@ func schema_pkg_apis_pingcap_v1alpha1_TiKVClient(ref common.ReferenceCallback) c } } -func schema_pkg_apis_pingcap_v1alpha1_TiKVSpec(ref common.ReferenceCallback) common.OpenAPIDefinition { +func schema_pkg_apis_pingcap_v1alpha1_TiKVConfig(ref common.ReferenceCallback) common.OpenAPIDefinition { return common.OpenAPIDefinition{ Schema: spec.Schema{ SchemaProps: spec.SchemaProps{ - Description: "TiKVSpec contains details of TiKV members", + Description: "TiKVConfig is the configuration of TiKV.", Type: []string{"object"}, Properties: map[string]spec.Schema{ - "replicas": { + "log-level": { SchemaProps: spec.SchemaProps{ - Type: []string{"integer"}, - Format: "int32", + Type: []string{"string"}, + Format: "", }, }, - "privileged": { + "log-file": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "log-rotation-timespan": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "panic-when-unexpected-key-or-data": { SchemaProps: spec.SchemaProps{ Type: []string{"boolean"}, Format: "", }, }, - "storageClassName": { + "addr": { SchemaProps: spec.SchemaProps{ Type: []string{"string"}, Format: "", }, }, - "maxFailoverCount": { + "advertise-addr": { SchemaProps: spec.SchemaProps{ - Type: []string{"integer"}, - Format: "int32", + Type: []string{"string"}, + Format: "", + }, + }, + "status-addr": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "status-thread-pool-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "grpc-compression-type": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "grpc-concurrency": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "grpc-concurrent-stream": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "grpc-raft-conn-num": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "grpc-stream-initial-window-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "grpc-keepalive-time": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "grpc-keepalive-timeout": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "concurrent-send-snap-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "concurrent-recv-snap-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "end-point-recursion-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "end-point-stream-channel-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "end-point-batch-row-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "end-point-stream-batch-row-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "end-point-enable-batch-if-possible": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "end-point-request-max-handle-duration": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "snap-max-write-bytes-per-sec": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "snap-max-total-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "stats-concurrency": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "heavy-load-threshold": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "heavy-load-wait-duration": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "labels": { + SchemaProps: spec.SchemaProps{ + Type: []string{"object"}, + AdditionalProperties: &spec.SchemaOrBool{ + Allows: true, + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + }, + }, + }, + "server": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVServerConfig"), + }, + }, + "storage": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVStorageConfig"), + }, + }, + "raftstore": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVRaftstoreConfig"), + }, + }, + "rocksdb": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVDbConfig"), + }, + }, + }, + }, + }, + Dependencies: []string{ + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVDbConfig", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVRaftstoreConfig", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVServerConfig", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVStorageConfig"}, + } +} + +func schema_pkg_apis_pingcap_v1alpha1_TiKVDbConfig(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "TiKVDbConfig is the rocksdb config.", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "wal-recovery-mode": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "wal-dir": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "wal-ttl-seconds": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "wal-size-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "max-total-wal-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "max-background-jobs": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "max-manifest-file-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "create-if-missing": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "max-open-files": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "enable-statistics": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "stats-dump-period": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "compaction-readahead-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "info-log-max-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "info-log-roll-time": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "info-log-keep-log-file-num": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "info-log-dir": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "rate-bytes-per-sec": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "rate-limiter-mode": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "auto-tuned": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "bytes-per-sync": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "wal-bytes-per-sync": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "max-sub-compactions": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "writable-file-max-buffer-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "use-direct-io-for-flush-and-compaction": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "enable-pipelined-write": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "defaultcf": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVCfConfig"), + }, + }, + "writecf": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVCfConfig"), + }, + }, + "lockcf": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVCfConfig"), + }, + }, + "raftcf": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVCfConfig"), + }, + }, + "titan": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVTitanDBConfig"), + }, + }, + }, + }, + }, + Dependencies: []string{ + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVCfConfig", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVTitanDBConfig"}, + } +} + +func schema_pkg_apis_pingcap_v1alpha1_TiKVRaftstoreConfig(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "TiKVRaftstoreConfig is the configuration of TiKV raftstore component.", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "sync-log": { + SchemaProps: spec.SchemaProps{ + Description: "true for high reliability, prevent data loss when power failure.", + Type: []string{"boolean"}, + Format: "", + }, + }, + "raft-base-tick-interval": { + SchemaProps: spec.SchemaProps{ + Description: "raft-base-tick-interval is a base tick interval (ms).", + Type: []string{"string"}, + Format: "", + }, + }, + "raft-heartbeat-ticks": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "raft-election-timeout-ticks": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "raft-entry-max-size": { + SchemaProps: spec.SchemaProps{ + Description: "When the entry exceed the max size, reject to propose it.", + Type: []string{"string"}, + Format: "", + }, + }, + "raft-log-gc-tick-interval": { + SchemaProps: spec.SchemaProps{ + Description: "Interval to gc unnecessary raft log (ms).", + Type: []string{"string"}, + Format: "", + }, + }, + "raft-log-gc-threshold": { + SchemaProps: spec.SchemaProps{ + Description: "A threshold to gc stale raft log, must >= 1.", + Type: []string{"integer"}, + Format: "int64", + }, + }, + "raft-log-gc-count-limit": { + SchemaProps: spec.SchemaProps{ + Description: "When entry count exceed this value, gc will be forced trigger.", + Type: []string{"integer"}, + Format: "int64", + }, + }, + "raft-log-gc-size-limit": { + SchemaProps: spec.SchemaProps{ + Description: "When the approximate size of raft log entries exceed this value gc will be forced trigger.", + Type: []string{"string"}, + Format: "", + }, + }, + "raft-entry-cache-life-time": { + SchemaProps: spec.SchemaProps{ + Description: "When a peer is not responding for this time, leader will not keep entry cache for it.", + Type: []string{"string"}, + Format: "", + }, + }, + "raft-reject-transfer-leader-duration": { + SchemaProps: spec.SchemaProps{ + Description: "When a peer is newly added, reject transferring leader to the peer for a while.", + Type: []string{"string"}, + Format: "", + }, + }, + "split-region-check-tick-interval": { + SchemaProps: spec.SchemaProps{ + Description: "Interval (ms) to check region whether need to be split or not.", + Type: []string{"string"}, + Format: "", + }, + }, + "region-split-check-diff": { + SchemaProps: spec.SchemaProps{ + Description: "/ When size change of region exceed the diff since last check, it / will be checked again whether it should be split.", + Type: []string{"string"}, + Format: "", + }, + }, + "region-compact-check-interval": { + SchemaProps: spec.SchemaProps{ + Description: "/ Interval (ms) to check whether start compaction for a region.", + Type: []string{"string"}, + Format: "", + }, + }, + "clean-stale-peer-delay": { + SchemaProps: spec.SchemaProps{ + Description: "delay time before deleting a stale peer", + Type: []string{"string"}, + Format: "", + }, + }, + "region-compact-check-step": { + SchemaProps: spec.SchemaProps{ + Description: "/ Number of regions for each time checking.", + Type: []string{"integer"}, + Format: "int64", + }, + }, + "region-compact-min-tombstones": { + SchemaProps: spec.SchemaProps{ + Description: "/ Minimum number of tombstones to trigger manual compaction.", + Type: []string{"integer"}, + Format: "int64", + }, + }, + "region-compact-tombstones-percent": { + SchemaProps: spec.SchemaProps{ + Description: "/ Minimum percentage of tombstones to trigger manual compaction. / Should between 1 and 100.", + Type: []string{"integer"}, + Format: "int64", + }, + }, + "pd-heartbeat-tick-interval": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "pd-store-heartbeat-tick-interval": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "snap-mgr-gc-tick-interval": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "snap-gc-timeout": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "lock-cf-compact-interval": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "lock-cf-compact-bytes-threshold": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "notify-capacity": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "messages-per-tick": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "max-peer-down-duration": { + SchemaProps: spec.SchemaProps{ + Description: "/ When a peer is not active for max-peer-down-duration / the peer is considered to be down and is reported to PD.", + Type: []string{"string"}, + Format: "", + }, + }, + "max-leader-missing-duration": { + SchemaProps: spec.SchemaProps{ + Description: "/ If the leader of a peer is missing for longer than max-leader-missing-duration / the peer would ask pd to confirm whether it is valid in any region. / If the peer is stale and is not valid in any region, it will destroy itself.", + Type: []string{"string"}, + Format: "", + }, + }, + "abnormal-leader-missing-duration": { + SchemaProps: spec.SchemaProps{ + Description: "/ Similar to the max-leader-missing-duration, instead it will log warnings and / try to alert monitoring systems, if there is any.", + Type: []string{"string"}, + Format: "", + }, + }, + "peer-stale-state-check-interval": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "leader-transfer-max-log-lag": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "snap-apply-batch-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "consistency-check-interval": { + SchemaProps: spec.SchemaProps{ + Description: "Interval (ms) to check region whether the data is consistent.", + Type: []string{"string"}, + Format: "", + }, + }, + "report-region-flow-interval": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "raft-store-max-leader-lease": { + SchemaProps: spec.SchemaProps{ + Description: "The lease provided by a successfully proposed and applied entry.", + Type: []string{"string"}, + Format: "", + }, + }, + "right-derive-when-split": { + SchemaProps: spec.SchemaProps{ + Description: "Right region derive origin region id when split.", + Type: []string{"boolean"}, + Format: "", + }, + }, + "allow-remove-leader": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "merge-max-log-gap": { + SchemaProps: spec.SchemaProps{ + Description: "/ Max log gap allowed to propose merge.", + Type: []string{"integer"}, + Format: "int64", + }, + }, + "merge-check-tick-interval": { + SchemaProps: spec.SchemaProps{ + Description: "/ Interval to re-propose merge.", + Type: []string{"string"}, + Format: "", + }, + }, + "use-delete-range": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "cleanup-import-sst-interval": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "apply-max-batch-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "apply-pool-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "store-max-batch-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "store-pool-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "hibernate-regions": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + }, + }, + }, + } +} + +func schema_pkg_apis_pingcap_v1alpha1_TiKVServerConfig(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "TiKVServerConfig is the configuration of TiKV server.", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "grpc-compression-type": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "grpc-concurrency": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "grpc-concurrent-stream": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "grpc-raft-conn-num": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "grpc-stream-initial-window-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "grpc-keepalive-time": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "grpc-keepalive-timeout": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "concurrent-send-snap-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "concurrent-recv-snap-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "end-point-recursion-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "end-point-stream-channel-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "end-point-batch-row-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "end-point-stream-batch-row-limit": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "end-point-enable-batch-if-possible": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "end-point-request-max-handle-duration": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "snap-max-write-bytes-per-sec": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "snap-max-total-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "stats-concurrency": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "heavy-load-threshold": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "heavy-load-wait-duration": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "labels": { + SchemaProps: spec.SchemaProps{ + Type: []string{"object"}, + AdditionalProperties: &spec.SchemaOrBool{ + Allows: true, + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + }, + }, + }, + }, + }, + }, + } +} + +func schema_pkg_apis_pingcap_v1alpha1_TiKVSpec(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "TiKVSpec contains details of TiKV members", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "replicas": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "privileged": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "storageClassName": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "maxFailoverCount": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int32", + }, + }, + "config": { + SchemaProps: spec.SchemaProps{ + Description: "Config is the Configuration of tikv-servers", + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVConfig"), }, }, }, Required: []string{"replicas"}, }, }, + Dependencies: []string{ + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVConfig"}, + } +} + +func schema_pkg_apis_pingcap_v1alpha1_TiKVStorageConfig(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "TiKVStorageConfig is the config of storage", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "data-dir": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "max-key-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "scheduler-notify-capacity": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "scheduler-concurrency": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "scheduler-worker-pool-size": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "scheduler-pending-write-threshold": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "block-cache": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVBlockCacheConfig"), + }, + }, + }, + }, + }, + Dependencies: []string{ + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiKVBlockCacheConfig"}, + } +} + +func schema_pkg_apis_pingcap_v1alpha1_TiKVTitanDBConfig(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "TiKVTitanDBConfig is the config a titian db.", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "enabled": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "dirname": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "disable-gc": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "max-background-gc": { + SchemaProps: spec.SchemaProps{ + Type: []string{"integer"}, + Format: "int64", + }, + }, + "purge-obsolete-files-period": { + SchemaProps: spec.SchemaProps{ + Description: "The value of this field will be truncated to seconds.", + Type: []string{"string"}, + Format: "", + }, + }, + }, + }, + }, } } diff --git a/pkg/apis/pingcap/v1alpha1/tikv_config.go b/pkg/apis/pingcap/v1alpha1/tikv_config.go index a9b6df450b..698464fbf3 100644 --- a/pkg/apis/pingcap/v1alpha1/tikv_config.go +++ b/pkg/apis/pingcap/v1alpha1/tikv_config.go @@ -19,326 +19,326 @@ package v1alpha1 // +k8s:openapi-gen=true type TiKVConfig struct { // +optional - LogLevel string `json:"log-level,omitempty",toml:"log-level,omitempty"` + LogLevel string `json:"log-level,omitempty" toml:"log-level,omitempty"` // +optional - LogFile string `json:"log-file,omitempty",toml:"log-file,omitempty"` + LogFile string `json:"log-file,omitempty" toml:"log-file,omitempty"` // +optional - LogRotationTimespan string `json:"log-rotation-timespan,omitempty",toml:"log-rotation-timespan,omitempty"` + LogRotationTimespan string `json:"log-rotation-timespan,omitempty" toml:"log-rotation-timespan,omitempty"` // +optional - PanicWhenUnexpectedKeyOrData *bool `json:"panic-when-unexpected-key-or-data,omitempty",toml:"panic-when-unexpected-key-or-data,omitempty"` + PanicWhenUnexpectedKeyOrData *bool `json:"panic-when-unexpected-key-or-data,omitempty" toml:"panic-when-unexpected-key-or-data,omitempty"` // +optional - Addr string `json:"addr,omitempty",toml:"addr,omitempty"` + Addr string `json:"addr,omitempty" toml:"addr,omitempty"` // +optional - AdvertiseAddr string `json:"advertise-addr,omitempty",toml:"advertise-addr,omitempty"` + AdvertiseAddr string `json:"advertise-addr,omitempty" toml:"advertise-addr,omitempty"` // +optional - StatusAddr string `json:"status-addr,omitempty",toml:"status-addr,omitempty"` + StatusAddr string `json:"status-addr,omitempty" toml:"status-addr,omitempty"` // +optional - StatusThreadPoolSize string `json:"status-thread-pool-size,omitempty",toml:"status-thread-pool-size,omitempty"` + StatusThreadPoolSize string `json:"status-thread-pool-size,omitempty" toml:"status-thread-pool-size,omitempty"` // +optional - GrpcCompressionType string `json:"grpc-compression-type,omitempty",toml:"grpc-compression-type,omitempty"` + GrpcCompressionType string `json:"grpc-compression-type,omitempty" toml:"grpc-compression-type,omitempty"` // +optional - GrpcConcurrency *uint `json:"grpc-concurrency,omitempty",toml:"grpc-concurrency,omitempty"` + GrpcConcurrency *uint `json:"grpc-concurrency,omitempty" toml:"grpc-concurrency,omitempty"` // +optional - GrpcConcurrentStream *uint `json:"grpc-concurrent-stream,omitempty",toml:"grpc-concurrent-stream,omitempty"` + GrpcConcurrentStream *uint `json:"grpc-concurrent-stream,omitempty" toml:"grpc-concurrent-stream,omitempty"` // +optional - GrpcRaftConnNum *uint `json:"grpc-raft-conn-num,omitempty",toml:"grpc-raft-conn-num,omitempty"` + GrpcRaftConnNum *uint `json:"grpc-raft-conn-num,omitempty" toml:"grpc-raft-conn-num,omitempty"` // +optional - GrpcStreamInitialWindowSize string `json:"grpc-stream-initial-window-size,omitempty",toml:"grpc-stream-initial-window-size,omitempty"` + GrpcStreamInitialWindowSize string `json:"grpc-stream-initial-window-size,omitempty" toml:"grpc-stream-initial-window-size,omitempty"` // +optional - GrpcKeepaliveTime string `json:"grpc-keepalive-time,omitempty",toml:"grpc-keepalive-time,omitempty"` + GrpcKeepaliveTime string `json:"grpc-keepalive-time,omitempty" toml:"grpc-keepalive-time,omitempty"` // +optional - GrpcKeepaliveTimeout string `json:"grpc-keepalive-timeout,omitempty",toml:"grpc-keepalive-timeout,omitempty"` + GrpcKeepaliveTimeout string `json:"grpc-keepalive-timeout,omitempty" toml:"grpc-keepalive-timeout,omitempty"` // +optional - ConcurrentSendSnapLimit *uint `json:"concurrent-send-snap-limit,omitempty",toml:"concurrent-send-snap-limit,omitempty"` + ConcurrentSendSnapLimit *uint `json:"concurrent-send-snap-limit,omitempty" toml:"concurrent-send-snap-limit,omitempty"` // +optional - ConcurrentRecvSnapLimit *uint `json:"concurrent-recv-snap-limit,omitempty",toml:"concurrent-recv-snap-limit,omitempty"` + ConcurrentRecvSnapLimit *uint `json:"concurrent-recv-snap-limit,omitempty" toml:"concurrent-recv-snap-limit,omitempty"` // +optional - EndPointRecursionLimit *uint `json:"end-point-recursion-limit,omitempty",toml:"end-point-recursion-limit,omitempty"` + EndPointRecursionLimit *uint `json:"end-point-recursion-limit,omitempty" toml:"end-point-recursion-limit,omitempty"` // +optional - EndPointStreamChannelSize *uint `json:"end-point-stream-channel-size,omitempty",toml:"end-point-stream-channel-size,omitempty"` + EndPointStreamChannelSize *uint `json:"end-point-stream-channel-size,omitempty" toml:"end-point-stream-channel-size,omitempty"` // +optional - EndPointBatchRowLimit *uint `json:"end-point-batch-row-limit,omitempty",toml:"end-point-batch-row-limit,omitempty"` + EndPointBatchRowLimit *uint `json:"end-point-batch-row-limit,omitempty" toml:"end-point-batch-row-limit,omitempty"` // +optional - EndPointStreamBatchRowLimit *uint `json:"end-point-stream-batch-row-limit,omitempty",toml:"end-point-stream-batch-row-limit,omitempty"` + EndPointStreamBatchRowLimit *uint `json:"end-point-stream-batch-row-limit,omitempty" toml:"end-point-stream-batch-row-limit,omitempty"` // +optional - EndPointEnableBatchIfPossible *uint `json:"end-point-enable-batch-if-possible,omitempty",toml:"end-point-enable-batch-if-possible,omitempty"` + EndPointEnableBatchIfPossible *uint `json:"end-point-enable-batch-if-possible,omitempty" toml:"end-point-enable-batch-if-possible,omitempty"` // +optional - EndPointRequestMaxHandleDuration string `json:"end-point-request-max-handle-duration,omitempty",toml:"end-point-request-max-handle-duration,omitempty"` + EndPointRequestMaxHandleDuration string `json:"end-point-request-max-handle-duration,omitempty" toml:"end-point-request-max-handle-duration,omitempty"` // +optional - SnapMaxWriteBytesPerSec string `json:"snap-max-write-bytes-per-sec,omitempty",toml:"snap-max-write-bytes-per-sec,omitempty"` + SnapMaxWriteBytesPerSec string `json:"snap-max-write-bytes-per-sec,omitempty" toml:"snap-max-write-bytes-per-sec,omitempty"` // +optional - SnapMaxTotalSize string `json:"snap-max-total-size,omitempty",toml:"snap-max-total-size,omitempty"` + SnapMaxTotalSize string `json:"snap-max-total-size,omitempty" toml:"snap-max-total-size,omitempty"` // +optional - StatsConcurrency *uint `json:"stats-concurrency,omitempty",toml:"stats-concurrency,omitempty"` + StatsConcurrency *uint `json:"stats-concurrency,omitempty" toml:"stats-concurrency,omitempty"` // +optional - HeavyLoadThreshold *uint `json:"heavy-load-threshold,omitempty",toml:"heavy-load-threshold,omitempty"` + HeavyLoadThreshold *uint `json:"heavy-load-threshold,omitempty" toml:"heavy-load-threshold,omitempty"` // +optional - HeavyLoadWaitDuration string `json:"heavy-load-wait-duration,omitempty",toml:"heavy-load-wait-duration,omitempty"` + HeavyLoadWaitDuration string `json:"heavy-load-wait-duration,omitempty" toml:"heavy-load-wait-duration,omitempty"` // +optional - Labels map[string]string `json:"labels,omitempty",toml:"labels,omitempty"` + Labels map[string]string `json:"labels,omitempty" toml:"labels,omitempty"` // +optional - Server *TiKVServerConfig `json:"server,omitempty",toml:"server,omitempty"` + Server *TiKVServerConfig `json:"server,omitempty" toml:"server,omitempty"` // +optional - Storage *TiKVStorageConfig `json:"storage,omitempty",toml:"storage,omitempty"` + Storage *TiKVStorageConfig `json:"storage,omitempty" toml:"storage,omitempty"` // +optional - Raftstore *TiKVRaftstoreConfig `json:"raftstore,omitempty",toml:"raftstore,omitempty"` + Raftstore *TiKVRaftstoreConfig `json:"raftstore,omitempty" toml:"raftstore,omitempty"` // +optional - Rocksdb *TiKVDbConfig `json:"rocksdb,omitempty",toml:"rocksdb,omitempty"` + Rocksdb *TiKVDbConfig `json:"rocksdb,omitempty" toml:"rocksdb,omitempty"` } // TiKVDbConfig is the rocksdb config. // +k8s:openapi-gen=true type TiKVDbConfig struct { // +optional - WalRecoveryMode *int64 `json:"wal-recovery-mode,omitempty",toml:"wal-recovery-mode,omitempty"` + WalRecoveryMode *int64 `json:"wal-recovery-mode,omitempty" toml:"wal-recovery-mode,omitempty"` // +optional - WalDir string `json:"wal-dir,omitempty",toml:"wal-dir,omitempty"` + WalDir string `json:"wal-dir,omitempty" toml:"wal-dir,omitempty"` // +optional - WalTTLSeconds *int64 `json:"wal-ttl-seconds,omitempty",toml:"wal-ttl-seconds,omitempty"` + WalTTLSeconds *int64 `json:"wal-ttl-seconds,omitempty" toml:"wal-ttl-seconds,omitempty"` // +optional - WalSizeLimit string `json:"wal-size-limit,omitempty",toml:"wal-size-limit,omitempty"` + WalSizeLimit string `json:"wal-size-limit,omitempty" toml:"wal-size-limit,omitempty"` // +optional - MaxTotalWalSize string `json:"max-total-wal-size,omitempty",toml:"max-total-wal-size,omitempty"` + MaxTotalWalSize string `json:"max-total-wal-size,omitempty" toml:"max-total-wal-size,omitempty"` // +optional - MaxBackgroundJobs *int64 `json:"max-background-jobs,omitempty",toml:"max-background-jobs,omitempty"` + MaxBackgroundJobs *int64 `json:"max-background-jobs,omitempty" toml:"max-background-jobs,omitempty"` // +optional - MaxManifestFileSize string `json:"max-manifest-file-size,omitempty",toml:"max-manifest-file-size,omitempty"` + MaxManifestFileSize string `json:"max-manifest-file-size,omitempty" toml:"max-manifest-file-size,omitempty"` // +optional - CreateIfMissing *bool `json:"create-if-missing,omitempty",toml:"create-if-missing,omitempty"` + CreateIfMissing *bool `json:"create-if-missing,omitempty" toml:"create-if-missing,omitempty"` // +optional - MaxOpenFiles *int64 `json:"max-open-files,omitempty",toml:"max-open-files,omitempty"` + MaxOpenFiles *int64 `json:"max-open-files,omitempty" toml:"max-open-files,omitempty"` // +optional - EnableStatistics *bool `json:"enable-statistics,omitempty",toml:"enable-statistics,omitempty"` + EnableStatistics *bool `json:"enable-statistics,omitempty" toml:"enable-statistics,omitempty"` // +optional - StatsDumpPeriod string `json:"stats-dump-period,omitempty",toml:"stats-dump-period,omitempty"` + StatsDumpPeriod string `json:"stats-dump-period,omitempty" toml:"stats-dump-period,omitempty"` // +optional - CompactionReadaheadSize string `json:"compaction-readahead-size,omitempty",toml:"compaction-readahead-size,omitempty"` + CompactionReadaheadSize string `json:"compaction-readahead-size,omitempty" toml:"compaction-readahead-size,omitempty"` // +optional - InfoLogMaxSize string `json:"info-log-max-size,omitempty",toml:"info-log-max-size,omitempty"` + InfoLogMaxSize string `json:"info-log-max-size,omitempty" toml:"info-log-max-size,omitempty"` // +optional - InfoLogRollTime string `json:"info-log-roll-time,omitempty",toml:"info-log-roll-time,omitempty"` + InfoLogRollTime string `json:"info-log-roll-time,omitempty" toml:"info-log-roll-time,omitempty"` // +optional - InfoLogKeepLogFileNum *int64 `json:"info-log-keep-log-file-num,omitempty",toml:"info-log-keep-log-file-num,omitempty"` + InfoLogKeepLogFileNum *int64 `json:"info-log-keep-log-file-num,omitempty" toml:"info-log-keep-log-file-num,omitempty"` // +optional - InfoLogDir string `json:"info-log-dir,omitempty",toml:"info-log-dir,omitempty"` + InfoLogDir string `json:"info-log-dir,omitempty" toml:"info-log-dir,omitempty"` // +optional - RateBytesPerSec string `json:"rate-bytes-per-sec,omitempty",toml:"rate-bytes-per-sec,omitempty"` + RateBytesPerSec string `json:"rate-bytes-per-sec,omitempty" toml:"rate-bytes-per-sec,omitempty"` // +optional - RateLimiterMode *int64 `json:"rate-limiter-mode,omitempty",toml:"rate-limiter-mode,omitempty"` + RateLimiterMode *int64 `json:"rate-limiter-mode,omitempty" toml:"rate-limiter-mode,omitempty"` // +optional - AutoTuned *bool `json:"auto-tuned,omitempty",toml:"auto-tuned,omitempty"` + AutoTuned *bool `json:"auto-tuned,omitempty" toml:"auto-tuned,omitempty"` // +optional - BytesPerSync string `json:"bytes-per-sync,omitempty",toml:"bytes-per-sync,omitempty"` + BytesPerSync string `json:"bytes-per-sync,omitempty" toml:"bytes-per-sync,omitempty"` // +optional - WalBytesPerSync string `json:"wal-bytes-per-sync,omitempty",toml:"wal-bytes-per-sync,omitempty"` + WalBytesPerSync string `json:"wal-bytes-per-sync,omitempty" toml:"wal-bytes-per-sync,omitempty"` // +optional - MaxSubCompactions *int64 `json:"max-sub-compactions,omitempty",toml:"max-sub-compactions,omitempty"` + MaxSubCompactions *int64 `json:"max-sub-compactions,omitempty" toml:"max-sub-compactions,omitempty"` // +optional - WritableFileMaxBufferSize string `json:"writable-file-max-buffer-size,omitempty",toml:"writable-file-max-buffer-size,omitempty"` + WritableFileMaxBufferSize string `json:"writable-file-max-buffer-size,omitempty" toml:"writable-file-max-buffer-size,omitempty"` // +optional - UseDirectIoForFlushAndCompaction *bool `json:"use-direct-io-for-flush-and-compaction,omitempty",toml:"use-direct-io-for-flush-and-compaction,omitempty"` + UseDirectIoForFlushAndCompaction *bool `json:"use-direct-io-for-flush-and-compaction,omitempty" toml:"use-direct-io-for-flush-and-compaction,omitempty"` // +optional - EnablePipelinedWrite *bool `json:"enable-pipelined-write,omitempty",toml:"enable-pipelined-write,omitempty"` + EnablePipelinedWrite *bool `json:"enable-pipelined-write,omitempty" toml:"enable-pipelined-write,omitempty"` // +optional - Defaultcf *TiKVCfConfig `json:"defaultcf,omitempty",toml:"defaultcf,omitempty"` + Defaultcf *TiKVCfConfig `json:"defaultcf,omitempty" toml:"defaultcf,omitempty"` // +optional - Writecf *TiKVCfConfig `json:"writecf,omitempty",toml:"writecf,omitempty"` + Writecf *TiKVCfConfig `json:"writecf,omitempty" toml:"writecf,omitempty"` // +optional - Lockcf *TiKVCfConfig `json:"lockcf,omitempty",toml:"lockcf,omitempty"` + Lockcf *TiKVCfConfig `json:"lockcf,omitempty" toml:"lockcf,omitempty"` // +optional - Raftcf *TiKVCfConfig `json:"raftcf,omitempty",toml:"raftcf,omitempty"` + Raftcf *TiKVCfConfig `json:"raftcf,omitempty" toml:"raftcf,omitempty"` // +optional - Titan *TiKVTitanDBConfig `json:"titan,omitempty",toml:"titan,omitempty"` + Titan *TiKVTitanDBConfig `json:"titan,omitempty" toml:"titan,omitempty"` } // TiKVCfConfig is the config of a cf // +k8s:openapi-gen=true type TiKVCfConfig struct { // +optional - BlockSize string `json:"block-size,omitempty",toml:"block-size,omitempty"` + BlockSize string `json:"block-size,omitempty" toml:"block-size,omitempty"` // +optional - BlockCacheSize string `json:"block-cache-size,omitempty",toml:"block-cache-size,omitempty"` + BlockCacheSize string `json:"block-cache-size,omitempty" toml:"block-cache-size,omitempty"` // +optional - DisableBlockCache *bool `json:"disable-block-cache,omitempty",toml:"disable-block-cache,omitempty"` + DisableBlockCache *bool `json:"disable-block-cache,omitempty" toml:"disable-block-cache,omitempty"` // +optional - CacheIndexAndFilterBlocks *bool `json:"cache-index-and-filter-blocks,omitempty",toml:"cache-index-and-filter-blocks,omitempty"` + CacheIndexAndFilterBlocks *bool `json:"cache-index-and-filter-blocks,omitempty" toml:"cache-index-and-filter-blocks,omitempty"` PinL0FilterAndIndexBlocks *bool `toml:"pin-l0-filter-and-index-blocks"` // +optional - UseBloomFilter *bool `json:"use-bloom-filter,omitempty",toml:"use-bloom-filter,omitempty"` + UseBloomFilter *bool `json:"use-bloom-filter,omitempty" toml:"use-bloom-filter,omitempty"` // +optional - OptimizeFiltersForHits *bool `json:"optimize-filters-for-hits,omitempty",toml:"optimize-filters-for-hits,omitempty"` + OptimizeFiltersForHits *bool `json:"optimize-filters-for-hits,omitempty" toml:"optimize-filters-for-hits,omitempty"` // +optional - WholeKeyFiltering *bool `json:"whole-key-filtering,omitempty",toml:"whole-key-filtering,omitempty"` + WholeKeyFiltering *bool `json:"whole-key-filtering,omitempty" toml:"whole-key-filtering,omitempty"` // +optional - BloomFilterBitsPerKey *int64 `json:"bloom-filter-bits-per-key,omitempty",toml:"bloom-filter-bits-per-key,omitempty"` + BloomFilterBitsPerKey *int64 `json:"bloom-filter-bits-per-key,omitempty" toml:"bloom-filter-bits-per-key,omitempty"` // +optional - BlockBasedBloomFilter *bool `json:"block-based-bloom-filter,omitempty",toml:"block-based-bloom-filter,omitempty"` + BlockBasedBloomFilter *bool `json:"block-based-bloom-filter,omitempty" toml:"block-based-bloom-filter,omitempty"` // +optional - ReadAmpBytesPerBit *int64 `json:"read-amp-bytes-per-bit,omitempty",toml:"read-amp-bytes-per-bit,omitempty"` + ReadAmpBytesPerBit *int64 `json:"read-amp-bytes-per-bit,omitempty" toml:"read-amp-bytes-per-bit,omitempty"` // +optional - CompressionPerLevel []string `json:"compression-per-level,omitempty",toml:"compression-per-level,omitempty"` + CompressionPerLevel []string `json:"compression-per-level,omitempty" toml:"compression-per-level,omitempty"` // +optional - WriteBufferSize string `json:"write-buffer-size,omitempty",toml:"write-buffer-size,omitempty"` + WriteBufferSize string `json:"write-buffer-size,omitempty" toml:"write-buffer-size,omitempty"` // +optional - MaxWriteBufferNumber *int64 `json:"max-write-buffer-number,omitempty",toml:"max-write-buffer-number,omitempty"` + MaxWriteBufferNumber *int64 `json:"max-write-buffer-number,omitempty" toml:"max-write-buffer-number,omitempty"` // +optional - MinWriteBufferNumberToMerge *int64 `json:"min-write-buffer-number-to-merge,omitempty",toml:"min-write-buffer-number-to-merge,omitempty"` + MinWriteBufferNumberToMerge *int64 `json:"min-write-buffer-number-to-merge,omitempty" toml:"min-write-buffer-number-to-merge,omitempty"` // +optional - MaxBytesForLevelBase string `json:"max-bytes-for-level-base,omitempty",toml:"max-bytes-for-level-base,omitempty"` + MaxBytesForLevelBase string `json:"max-bytes-for-level-base,omitempty" toml:"max-bytes-for-level-base,omitempty"` // +optional - TargetFileSizeBase string `json:"target-file-size-base,omitempty",toml:"target-file-size-base,omitempty"` + TargetFileSizeBase string `json:"target-file-size-base,omitempty" toml:"target-file-size-base,omitempty"` Level0FileNumCompactionTrigger *int64 `toml:"level0-file-num-compaction-trigger"` Level0SlowdownWritesTrigger *int64 `toml:"level0-slowdown-writes-trigger"` Level0StopWritesTrigger *int64 `toml:"level0-stop-writes-trigger"` // +optional - MaxCompactionBytes string `json:"max-compaction-bytes,omitempty",toml:"max-compaction-bytes,omitempty"` + MaxCompactionBytes string `json:"max-compaction-bytes,omitempty" toml:"max-compaction-bytes,omitempty"` // +optional - CompactionPri *int64 `json:"compaction-pri,omitempty",toml:"compaction-pri,omitempty"` + CompactionPri *int64 `json:"compaction-pri,omitempty" toml:"compaction-pri,omitempty"` // +optional - DynamicLevelBytes *bool `json:"dynamic-level-bytes,omitempty",toml:"dynamic-level-bytes,omitempty"` + DynamicLevelBytes *bool `json:"dynamic-level-bytes,omitempty" toml:"dynamic-level-bytes,omitempty"` // +optional - NumLevels *int64 `json:"num-levels,omitempty",toml:"num-levels,omitempty"` + NumLevels *int64 `json:"num-levels,omitempty" toml:"num-levels,omitempty"` // +optional - MaxBytesForLevelMultiplier *int64 `json:"max-bytes-for-level-multiplier,omitempty",toml:"max-bytes-for-level-multiplier,omitempty"` + MaxBytesForLevelMultiplier *int64 `json:"max-bytes-for-level-multiplier,omitempty" toml:"max-bytes-for-level-multiplier,omitempty"` // +optional - CompactionStyle *int64 `json:"compaction-style,omitempty",toml:"compaction-style,omitempty"` + CompactionStyle *int64 `json:"compaction-style,omitempty" toml:"compaction-style,omitempty"` // +optional - DisableAutoCompactions *bool `json:"disable-auto-compactions,omitempty",toml:"disable-auto-compactions,omitempty"` + DisableAutoCompactions *bool `json:"disable-auto-compactions,omitempty" toml:"disable-auto-compactions,omitempty"` // +optional - SoftPendingCompactionBytesLimit string `json:"soft-pending-compaction-bytes-limit,omitempty",toml:"soft-pending-compaction-bytes-limit,omitempty"` + SoftPendingCompactionBytesLimit string `json:"soft-pending-compaction-bytes-limit,omitempty" toml:"soft-pending-compaction-bytes-limit,omitempty"` // +optional - HardPendingCompactionBytesLimit string `json:"hard-pending-compaction-bytes-limit,omitempty",toml:"hard-pending-compaction-bytes-limit,omitempty"` + HardPendingCompactionBytesLimit string `json:"hard-pending-compaction-bytes-limit,omitempty" toml:"hard-pending-compaction-bytes-limit,omitempty"` // +optional - ForceConsistencyChecks *bool `json:"force-consistency-checks,omitempty",toml:"force-consistency-checks,omitempty"` + ForceConsistencyChecks *bool `json:"force-consistency-checks,omitempty" toml:"force-consistency-checks,omitempty"` // +optional - PropSizeIndexDistance *int64 `json:"prop-size-index-distance,omitempty",toml:"prop-size-index-distance,omitempty"` + PropSizeIndexDistance *int64 `json:"prop-size-index-distance,omitempty" toml:"prop-size-index-distance,omitempty"` // +optional - PropKeysIndexDistance *int64 `json:"prop-keys-index-distance,omitempty",toml:"prop-keys-index-distance,omitempty"` + PropKeysIndexDistance *int64 `json:"prop-keys-index-distance,omitempty" toml:"prop-keys-index-distance,omitempty"` // +optional - EnableDoublySkiplist *bool `json:"enable-doubly-skiplist,omitempty",toml:"enable-doubly-skiplist,omitempty"` + EnableDoublySkiplist *bool `json:"enable-doubly-skiplist,omitempty" toml:"enable-doubly-skiplist,omitempty"` // +optional - Titan *TiKVTitanCfConfig `json:"titan,omitempty",toml:"titan,omitempty"` + Titan *TiKVTitanCfConfig `json:"titan,omitempty" toml:"titan,omitempty"` } // TiKVTitanCfConfig is the titian config. type TiKVTitanCfConfig struct { // +optional - MinBlobSize string `json:"min-blob-size,omitempty",toml:"min-blob-size,omitempty"` + MinBlobSize string `json:"min-blob-size,omitempty" toml:"min-blob-size,omitempty"` // +optional - BlobFileCompression string `json:"blob-file-compression,omitempty",toml:"blob-file-compression,omitempty"` + BlobFileCompression string `json:"blob-file-compression,omitempty" toml:"blob-file-compression,omitempty"` // +optional - BlobCacheSize string `json:"blob-cache-size,omitempty",toml:"blob-cache-size,omitempty"` + BlobCacheSize string `json:"blob-cache-size,omitempty" toml:"blob-cache-size,omitempty"` // +optional - MinGcBatchSize string `json:"min-gc-batch-size,omitempty",toml:"min-gc-batch-size,omitempty"` + MinGcBatchSize string `json:"min-gc-batch-size,omitempty" toml:"min-gc-batch-size,omitempty"` // +optional - MaxGcBatchSize string `json:"max-gc-batch-size,omitempty",toml:"max-gc-batch-size,omitempty"` + MaxGcBatchSize string `json:"max-gc-batch-size,omitempty" toml:"max-gc-batch-size,omitempty"` // +optional - DiscardableRatio float64 `json:"discardable-ratio,omitempty",toml:"discardable-ratio,omitempty"` + DiscardableRatio float64 `json:"discardable-ratio,omitempty" toml:"discardable-ratio,omitempty"` // +optional - SampleRatio float64 `json:"sample-ratio,omitempty",toml:"sample-ratio,omitempty"` + SampleRatio float64 `json:"sample-ratio,omitempty" toml:"sample-ratio,omitempty"` // +optional - MergeSmallFileThreshold string `json:"merge-small-file-threshold,omitempty",toml:"merge-small-file-threshold,omitempty"` + MergeSmallFileThreshold string `json:"merge-small-file-threshold,omitempty" toml:"merge-small-file-threshold,omitempty"` // +optional - BlobRunMode string `json:"blob-run-mode,omitempty",toml:"blob-run-mode,omitempty"` + BlobRunMode string `json:"blob-run-mode,omitempty" toml:"blob-run-mode,omitempty"` } // TiKVTitanDBConfig is the config a titian db. // +k8s:openapi-gen=true type TiKVTitanDBConfig struct { // +optional - Enabled *bool `json:"enabled,omitempty",toml:"enabled,omitempty"` + Enabled *bool `json:"enabled,omitempty" toml:"enabled,omitempty"` // +optional - Dirname string `json:"dirname,omitempty",toml:"dirname,omitempty"` + Dirname string `json:"dirname,omitempty" toml:"dirname,omitempty"` // +optional - DisableGc *bool `json:"disable-gc,omitempty",toml:"disable-gc,omitempty"` + DisableGc *bool `json:"disable-gc,omitempty" toml:"disable-gc,omitempty"` // +optional - MaxBackgroundGc *int64 `json:"max-background-gc,omitempty",toml:"max-background-gc,omitempty"` + MaxBackgroundGc *int64 `json:"max-background-gc,omitempty" toml:"max-background-gc,omitempty"` // The value of this field will be truncated to seconds. // +optional - PurgeObsoleteFilesPeriod string `json:"purge-obsolete-files-period,omitempty",toml:"purge-obsolete-files-period,omitempty"` + PurgeObsoleteFilesPeriod string `json:"purge-obsolete-files-period,omitempty" toml:"purge-obsolete-files-period,omitempty"` } // TiKVStorageConfig is the config of storage // +k8s:openapi-gen=true type TiKVStorageConfig struct { // +optional - DataDir string `json:"data-dir,omitempty",toml:"data-dir,omitempty"` + DataDir string `json:"data-dir,omitempty" toml:"data-dir,omitempty"` // +optional - MaxKeySize *int64 `json:"max-key-size,omitempty",toml:"max-key-size,omitempty"` + MaxKeySize *int64 `json:"max-key-size,omitempty" toml:"max-key-size,omitempty"` // +optional - SchedulerNotifyCapacity *int64 `json:"scheduler-notify-capacity,omitempty",toml:"scheduler-notify-capacity,omitempty"` + SchedulerNotifyCapacity *int64 `json:"scheduler-notify-capacity,omitempty" toml:"scheduler-notify-capacity,omitempty"` // +optional - SchedulerConcurrency *int64 `json:"scheduler-concurrency,omitempty",toml:"scheduler-concurrency,omitempty"` + SchedulerConcurrency *int64 `json:"scheduler-concurrency,omitempty" toml:"scheduler-concurrency,omitempty"` // +optional - SchedulerWorkerPoolSize *int64 `json:"scheduler-worker-pool-size,omitempty",toml:"scheduler-worker-pool-size,omitempty"` + SchedulerWorkerPoolSize *int64 `json:"scheduler-worker-pool-size,omitempty" toml:"scheduler-worker-pool-size,omitempty"` // +optional - SchedulerPendingWriteThreshold string `json:"scheduler-pending-write-threshold,omitempty",toml:"scheduler-pending-write-threshold,omitempty"` + SchedulerPendingWriteThreshold string `json:"scheduler-pending-write-threshold,omitempty" toml:"scheduler-pending-write-threshold,omitempty"` // +optional - BlockCache *TiKVBlockCacheConfig `json:"block-cache,omitempty",toml:"block-cache,omitempty"` + BlockCache *TiKVBlockCacheConfig `json:"block-cache,omitempty" toml:"block-cache,omitempty"` } // TiKVBlockCacheConfig is the config of a block cache // +k8s:openapi-gen=true type TiKVBlockCacheConfig struct { // +optional - Shared *bool `json:"shared,omitempty",toml:"shared,omitempty"` + Shared *bool `json:"shared,omitempty" toml:"shared,omitempty"` // +optional - Capacity string `json:"capacity,omitempty",toml:"capacity,omitempty"` + Capacity string `json:"capacity,omitempty" toml:"capacity,omitempty"` // +optional - NumShardBits *int64 `json:"num-shard-bits,omitempty",toml:"num-shard-bits,omitempty"` + NumShardBits *int64 `json:"num-shard-bits,omitempty" toml:"num-shard-bits,omitempty"` // +optional - StrictCapacityLimit *bool `json:"strict-capacity-limit,omitempty",toml:"strict-capacity-limit,omitempty"` + StrictCapacityLimit *bool `json:"strict-capacity-limit,omitempty" toml:"strict-capacity-limit,omitempty"` // +optional - HighPriPoolRatio float64 `json:"high-pri-pool-ratio,omitempty",toml:"high-pri-pool-ratio,omitempty"` + HighPriPoolRatio float64 `json:"high-pri-pool-ratio,omitempty" toml:"high-pri-pool-ratio,omitempty"` // +optional - MemoryAllocator string `json:"memory-allocator,omitempty",toml:"memory-allocator,omitempty"` + MemoryAllocator string `json:"memory-allocator,omitempty" toml:"memory-allocator,omitempty"` } // TiKVServerConfig is the configuration of TiKV server. // +k8s:openapi-gen=true type TiKVServerConfig struct { // +optional - GrpcCompressionType string `json:"grpc-compression-type,omitempty",toml:"grpc-compression-type,omitempty"` + GrpcCompressionType string `json:"grpc-compression-type,omitempty" toml:"grpc-compression-type,omitempty"` // +optional - GrpcConcurrency *int64 `json:"grpc-concurrency,omitempty",toml:"grpc-concurrency,omitempty"` + GrpcConcurrency *int64 `json:"grpc-concurrency,omitempty" toml:"grpc-concurrency,omitempty"` // +optional - GrpcConcurrentStream *int64 `json:"grpc-concurrent-stream,omitempty",toml:"grpc-concurrent-stream,omitempty"` + GrpcConcurrentStream *int64 `json:"grpc-concurrent-stream,omitempty" toml:"grpc-concurrent-stream,omitempty"` // +optional - GrpcRaftConnNum *int64 `json:"grpc-raft-conn-num,omitempty",toml:"grpc-raft-conn-num,omitempty"` + GrpcRaftConnNum *int64 `json:"grpc-raft-conn-num,omitempty" toml:"grpc-raft-conn-num,omitempty"` // +optional - GrpcStreamInitialWindowSize string `json:"grpc-stream-initial-window-size,omitempty",toml:"grpc-stream-initial-window-size,omitempty"` + GrpcStreamInitialWindowSize string `json:"grpc-stream-initial-window-size,omitempty" toml:"grpc-stream-initial-window-size,omitempty"` // +optional - GrpcKeepaliveTime string `json:"grpc-keepalive-time,omitempty",toml:"grpc-keepalive-time,omitempty"` + GrpcKeepaliveTime string `json:"grpc-keepalive-time,omitempty" toml:"grpc-keepalive-time,omitempty"` // +optional - GrpcKeepaliveTimeout string `json:"grpc-keepalive-timeout,omitempty",toml:"grpc-keepalive-timeout,omitempty"` + GrpcKeepaliveTimeout string `json:"grpc-keepalive-timeout,omitempty" toml:"grpc-keepalive-timeout,omitempty"` // +optional - ConcurrentSendSnapLimit *int64 `json:"concurrent-send-snap-limit,omitempty",toml:"concurrent-send-snap-limit,omitempty"` + ConcurrentSendSnapLimit *int64 `json:"concurrent-send-snap-limit,omitempty" toml:"concurrent-send-snap-limit,omitempty"` // +optional - ConcurrentRecvSnapLimit *int64 `json:"concurrent-recv-snap-limit,omitempty",toml:"concurrent-recv-snap-limit,omitempty"` + ConcurrentRecvSnapLimit *int64 `json:"concurrent-recv-snap-limit,omitempty" toml:"concurrent-recv-snap-limit,omitempty"` // +optional - EndPointRecursionLimit *int64 `json:"end-point-recursion-limit,omitempty",toml:"end-point-recursion-limit,omitempty"` + EndPointRecursionLimit *int64 `json:"end-point-recursion-limit,omitempty" toml:"end-point-recursion-limit,omitempty"` // +optional - EndPointStreamChannelSize *int64 `json:"end-point-stream-channel-size,omitempty",toml:"end-point-stream-channel-size,omitempty"` + EndPointStreamChannelSize *int64 `json:"end-point-stream-channel-size,omitempty" toml:"end-point-stream-channel-size,omitempty"` // +optional - EndPointBatchRowLimit *int64 `json:"end-point-batch-row-limit,omitempty",toml:"end-point-batch-row-limit,omitempty"` + EndPointBatchRowLimit *int64 `json:"end-point-batch-row-limit,omitempty" toml:"end-point-batch-row-limit,omitempty"` // +optional - EndPointStreamBatchRowLimit *int64 `json:"end-point-stream-batch-row-limit,omitempty",toml:"end-point-stream-batch-row-limit,omitempty"` + EndPointStreamBatchRowLimit *int64 `json:"end-point-stream-batch-row-limit,omitempty" toml:"end-point-stream-batch-row-limit,omitempty"` // +optional - EndPointEnableBatchIfPossible *bool `json:"end-point-enable-batch-if-possible,omitempty",toml:"end-point-enable-batch-if-possible,omitempty"` + EndPointEnableBatchIfPossible *bool `json:"end-point-enable-batch-if-possible,omitempty" toml:"end-point-enable-batch-if-possible,omitempty"` // +optional - EndPointRequestMaxHandleDuration string `json:"end-point-request-max-handle-duration,omitempty",toml:"end-point-request-max-handle-duration,omitempty"` + EndPointRequestMaxHandleDuration string `json:"end-point-request-max-handle-duration,omitempty" toml:"end-point-request-max-handle-duration,omitempty"` // +optional - SnapMaxWriteBytesPerSec string `json:"snap-max-write-bytes-per-sec,omitempty",toml:"snap-max-write-bytes-per-sec,omitempty"` + SnapMaxWriteBytesPerSec string `json:"snap-max-write-bytes-per-sec,omitempty" toml:"snap-max-write-bytes-per-sec,omitempty"` // +optional - SnapMaxTotalSize string `json:"snap-max-total-size,omitempty",toml:"snap-max-total-size,omitempty"` + SnapMaxTotalSize string `json:"snap-max-total-size,omitempty" toml:"snap-max-total-size,omitempty"` // +optional - StatsConcurrency *int64 `json:"stats-concurrency,omitempty",toml:"stats-concurrency,omitempty"` + StatsConcurrency *int64 `json:"stats-concurrency,omitempty" toml:"stats-concurrency,omitempty"` // +optional - HeavyLoadThreshold *int64 `json:"heavy-load-threshold,omitempty",toml:"heavy-load-threshold,omitempty"` + HeavyLoadThreshold *int64 `json:"heavy-load-threshold,omitempty" toml:"heavy-load-threshold,omitempty"` // +optional - HeavyLoadWaitDuration string `json:"heavy-load-wait-duration,omitempty",toml:"heavy-load-wait-duration,omitempty"` + HeavyLoadWaitDuration string `json:"heavy-load-wait-duration,omitempty" toml:"heavy-load-wait-duration,omitempty"` // +optional - Labels map[string]string `json:"labels,omitempty",toml:"labels,omitempty"` + Labels map[string]string `json:"labels,omitempty" toml:"labels,omitempty"` } // TiKVRaftstoreConfig is the configuration of TiKV raftstore component. @@ -346,143 +346,143 @@ type TiKVServerConfig struct { type TiKVRaftstoreConfig struct { // true for high reliability, prevent data loss when power failure. // +optional - SyncLog *bool `json:"sync-log,omitempty",toml:"sync-log,omitempty"` + SyncLog *bool `json:"sync-log,omitempty" toml:"sync-log,omitempty"` // raft-base-tick-interval is a base tick interval (ms). // +optional - RaftBaseTickInterval string `json:"raft-base-tick-interval,omitempty",toml:"raft-base-tick-interval,omitempty"` + RaftBaseTickInterval string `json:"raft-base-tick-interval,omitempty" toml:"raft-base-tick-interval,omitempty"` // +optional - RaftHeartbeatTicks *int64 `json:"raft-heartbeat-ticks,omitempty",toml:"raft-heartbeat-ticks,omitempty"` + RaftHeartbeatTicks *int64 `json:"raft-heartbeat-ticks,omitempty" toml:"raft-heartbeat-ticks,omitempty"` // +optional - RaftElectionTimeoutTicks *int64 `json:"raft-election-timeout-ticks,omitempty",toml:"raft-election-timeout-ticks,omitempty"` + RaftElectionTimeoutTicks *int64 `json:"raft-election-timeout-ticks,omitempty" toml:"raft-election-timeout-ticks,omitempty"` // When the entry exceed the max size, reject to propose it. // +optional - RaftEntryMaxSize string `json:"raft-entry-max-size,omitempty",toml:"raft-entry-max-size,omitempty"` + RaftEntryMaxSize string `json:"raft-entry-max-size,omitempty" toml:"raft-entry-max-size,omitempty"` // Interval to gc unnecessary raft log (ms). // +optional - RaftLogGCTickInterval string `json:"raft-log-gc-tick-interval,omitempty",toml:"raft-log-gc-tick-interval,omitempty"` + RaftLogGCTickInterval string `json:"raft-log-gc-tick-interval,omitempty" toml:"raft-log-gc-tick-interval,omitempty"` // A threshold to gc stale raft log, must >= 1. // +optional - RaftLogGCThreshold *int64 `json:"raft-log-gc-threshold,omitempty",toml:"raft-log-gc-threshold,omitempty"` + RaftLogGCThreshold *int64 `json:"raft-log-gc-threshold,omitempty" toml:"raft-log-gc-threshold,omitempty"` // When entry count exceed this value, gc will be forced trigger. // +optional - RaftLogGCCountLimit *int64 `json:"raft-log-gc-count-limit,omitempty",toml:"raft-log-gc-count-limit,omitempty"` + RaftLogGCCountLimit *int64 `json:"raft-log-gc-count-limit,omitempty" toml:"raft-log-gc-count-limit,omitempty"` // When the approximate size of raft log entries exceed this value // gc will be forced trigger. // +optional - RaftLogGCSizeLimit string `json:"raft-log-gc-size-limit,omitempty",toml:"raft-log-gc-size-limit,omitempty"` + RaftLogGCSizeLimit string `json:"raft-log-gc-size-limit,omitempty" toml:"raft-log-gc-size-limit,omitempty"` // When a peer is not responding for this time, leader will not keep entry cache for it. // +optional - RaftEntryCacheLifeTime string `json:"raft-entry-cache-life-time,omitempty",toml:"raft-entry-cache-life-time,omitempty"` + RaftEntryCacheLifeTime string `json:"raft-entry-cache-life-time,omitempty" toml:"raft-entry-cache-life-time,omitempty"` // When a peer is newly added, reject transferring leader to the peer for a while. // +optional - RaftRejectTransferLeaderDuration string `json:"raft-reject-transfer-leader-duration,omitempty",toml:"raft-reject-transfer-leader-duration,omitempty"` + RaftRejectTransferLeaderDuration string `json:"raft-reject-transfer-leader-duration,omitempty" toml:"raft-reject-transfer-leader-duration,omitempty"` // Interval (ms) to check region whether need to be split or not. // +optional - SplitRegionCheckTickInterval string `json:"split-region-check-tick-interval,omitempty",toml:"split-region-check-tick-interval,omitempty"` + SplitRegionCheckTickInterval string `json:"split-region-check-tick-interval,omitempty" toml:"split-region-check-tick-interval,omitempty"` /// When size change of region exceed the diff since last check, it /// will be checked again whether it should be split. // +optional - RegionSplitCheckDiff string `json:"region-split-check-diff,omitempty",toml:"region-split-check-diff,omitempty"` + RegionSplitCheckDiff string `json:"region-split-check-diff,omitempty" toml:"region-split-check-diff,omitempty"` /// Interval (ms) to check whether start compaction for a region. // +optional - RegionCompactCheckInterval string `json:"region-compact-check-interval,omitempty",toml:"region-compact-check-interval,omitempty"` + RegionCompactCheckInterval string `json:"region-compact-check-interval,omitempty" toml:"region-compact-check-interval,omitempty"` // delay time before deleting a stale peer // +optional - CleanStalePeerDelay string `json:"clean-stale-peer-delay,omitempty",toml:"clean-stale-peer-delay,omitempty"` + CleanStalePeerDelay string `json:"clean-stale-peer-delay,omitempty" toml:"clean-stale-peer-delay,omitempty"` /// Number of regions for each time checking. // +optional - RegionCompactCheckStep *int64 `json:"region-compact-check-step,omitempty",toml:"region-compact-check-step,omitempty"` + RegionCompactCheckStep *int64 `json:"region-compact-check-step,omitempty" toml:"region-compact-check-step,omitempty"` /// Minimum number of tombstones to trigger manual compaction. // +optional - RegionCompactMinTombstones *int64 `json:"region-compact-min-tombstones,omitempty",toml:"region-compact-min-tombstones,omitempty"` + RegionCompactMinTombstones *int64 `json:"region-compact-min-tombstones,omitempty" toml:"region-compact-min-tombstones,omitempty"` /// Minimum percentage of tombstones to trigger manual compaction. /// Should between 1 and 100. // +optional - RegionCompactTombstonesPercent *int64 `json:"region-compact-tombstones-percent,omitempty",toml:"region-compact-tombstones-percent,omitempty"` + RegionCompactTombstonesPercent *int64 `json:"region-compact-tombstones-percent,omitempty" toml:"region-compact-tombstones-percent,omitempty"` // +optional - PdHeartbeatTickInterval string `json:"pd-heartbeat-tick-interval,omitempty",toml:"pd-heartbeat-tick-interval,omitempty"` + PdHeartbeatTickInterval string `json:"pd-heartbeat-tick-interval,omitempty" toml:"pd-heartbeat-tick-interval,omitempty"` // +optional - PdStoreHeartbeatTickInterval string `json:"pd-store-heartbeat-tick-interval,omitempty",toml:"pd-store-heartbeat-tick-interval,omitempty"` + PdStoreHeartbeatTickInterval string `json:"pd-store-heartbeat-tick-interval,omitempty" toml:"pd-store-heartbeat-tick-interval,omitempty"` // +optional - SnapMgrGCTickInterval string `json:"snap-mgr-gc-tick-interval,omitempty",toml:"snap-mgr-gc-tick-interval,omitempty"` + SnapMgrGCTickInterval string `json:"snap-mgr-gc-tick-interval,omitempty" toml:"snap-mgr-gc-tick-interval,omitempty"` // +optional - SnapGCTimeout string `json:"snap-gc-timeout,omitempty",toml:"snap-gc-timeout,omitempty"` + SnapGCTimeout string `json:"snap-gc-timeout,omitempty" toml:"snap-gc-timeout,omitempty"` // +optional - LockCfCompactInterval string `json:"lock-cf-compact-interval,omitempty",toml:"lock-cf-compact-interval,omitempty"` + LockCfCompactInterval string `json:"lock-cf-compact-interval,omitempty" toml:"lock-cf-compact-interval,omitempty"` // +optional - LockCfCompactBytesThreshold string `json:"lock-cf-compact-bytes-threshold,omitempty",toml:"lock-cf-compact-bytes-threshold,omitempty"` + LockCfCompactBytesThreshold string `json:"lock-cf-compact-bytes-threshold,omitempty" toml:"lock-cf-compact-bytes-threshold,omitempty"` // +optional - NotifyCapacity *int64 `json:"notify-capacity,omitempty",toml:"notify-capacity,omitempty"` + NotifyCapacity *int64 `json:"notify-capacity,omitempty" toml:"notify-capacity,omitempty"` // +optional - MessagesPerTick *int64 `json:"messages-per-tick,omitempty",toml:"messages-per-tick,omitempty"` + MessagesPerTick *int64 `json:"messages-per-tick,omitempty" toml:"messages-per-tick,omitempty"` /// When a peer is not active for max-peer-down-duration /// the peer is considered to be down and is reported to PD. // +optional - MaxPeerDownDuration string `json:"max-peer-down-duration,omitempty",toml:"max-peer-down-duration,omitempty"` + MaxPeerDownDuration string `json:"max-peer-down-duration,omitempty" toml:"max-peer-down-duration,omitempty"` /// If the leader of a peer is missing for longer than max-leader-missing-duration /// the peer would ask pd to confirm whether it is valid in any region. /// If the peer is stale and is not valid in any region, it will destroy itself. // +optional - MaxLeaderMissingDuration string `json:"max-leader-missing-duration,omitempty",toml:"max-leader-missing-duration,omitempty"` + MaxLeaderMissingDuration string `json:"max-leader-missing-duration,omitempty" toml:"max-leader-missing-duration,omitempty"` /// Similar to the max-leader-missing-duration, instead it will log warnings and /// try to alert monitoring systems, if there is any. // +optional - AbnormalLeaderMissingDuration string `json:"abnormal-leader-missing-duration,omitempty",toml:"abnormal-leader-missing-duration,omitempty"` + AbnormalLeaderMissingDuration string `json:"abnormal-leader-missing-duration,omitempty" toml:"abnormal-leader-missing-duration,omitempty"` // +optional - PeerStaleStateCheckInterval string `json:"peer-stale-state-check-interval,omitempty",toml:"peer-stale-state-check-interval,omitempty"` + PeerStaleStateCheckInterval string `json:"peer-stale-state-check-interval,omitempty" toml:"peer-stale-state-check-interval,omitempty"` // +optional - LeaderTransferMaxLogLag *int64 `json:"leader-transfer-max-log-lag,omitempty",toml:"leader-transfer-max-log-lag,omitempty"` + LeaderTransferMaxLogLag *int64 `json:"leader-transfer-max-log-lag,omitempty" toml:"leader-transfer-max-log-lag,omitempty"` // +optional - SnapApplyBatchSize string `json:"snap-apply-batch-size,omitempty",toml:"snap-apply-batch-size,omitempty"` + SnapApplyBatchSize string `json:"snap-apply-batch-size,omitempty" toml:"snap-apply-batch-size,omitempty"` // Interval (ms) to check region whether the data is consistent. // +optional - ConsistencyCheckInterval string `json:"consistency-check-interval,omitempty",toml:"consistency-check-interval,omitempty"` + ConsistencyCheckInterval string `json:"consistency-check-interval,omitempty" toml:"consistency-check-interval,omitempty"` // +optional - ReportRegionFlowInterval string `json:"report-region-flow-interval,omitempty",toml:"report-region-flow-interval,omitempty"` + ReportRegionFlowInterval string `json:"report-region-flow-interval,omitempty" toml:"report-region-flow-interval,omitempty"` // The lease provided by a successfully proposed and applied entry. // +optional - RaftStoreMaxLeaderLease string `json:"raft-store-max-leader-lease,omitempty",toml:"raft-store-max-leader-lease,omitempty"` + RaftStoreMaxLeaderLease string `json:"raft-store-max-leader-lease,omitempty" toml:"raft-store-max-leader-lease,omitempty"` // Right region derive origin region id when split. // +optional - RightDeriveWhenSplit *bool `json:"right-derive-when-split,omitempty",toml:"right-derive-when-split,omitempty"` + RightDeriveWhenSplit *bool `json:"right-derive-when-split,omitempty" toml:"right-derive-when-split,omitempty"` // +optional - AllowRemoveLeader *bool `json:"allow-remove-leader,omitempty",toml:"allow-remove-leader,omitempty"` + AllowRemoveLeader *bool `json:"allow-remove-leader,omitempty" toml:"allow-remove-leader,omitempty"` /// Max log gap allowed to propose merge. // +optional - MergeMaxLogGap *int64 `json:"merge-max-log-gap,omitempty",toml:"merge-max-log-gap,omitempty"` + MergeMaxLogGap *int64 `json:"merge-max-log-gap,omitempty" toml:"merge-max-log-gap,omitempty"` /// Interval to re-propose merge. // +optional - MergeCheckTickInterval string `json:"merge-check-tick-interval,omitempty",toml:"merge-check-tick-interval,omitempty"` + MergeCheckTickInterval string `json:"merge-check-tick-interval,omitempty" toml:"merge-check-tick-interval,omitempty"` // +optional - UseDeleteRange *bool `json:"use-delete-range,omitempty",toml:"use-delete-range,omitempty"` + UseDeleteRange *bool `json:"use-delete-range,omitempty" toml:"use-delete-range,omitempty"` // +optional - CleanupImportSstInterval string `json:"cleanup-import-sst-interval,omitempty",toml:"cleanup-import-sst-interval,omitempty"` + CleanupImportSstInterval string `json:"cleanup-import-sst-interval,omitempty" toml:"cleanup-import-sst-interval,omitempty"` // +optional - ApplyMaxBatchSize *int64 `json:"apply-max-batch-size,omitempty",toml:"apply-max-batch-size,omitempty"` + ApplyMaxBatchSize *int64 `json:"apply-max-batch-size,omitempty" toml:"apply-max-batch-size,omitempty"` // +optional - ApplyPoolSize *int64 `json:"apply-pool-size,omitempty",toml:"apply-pool-size,omitempty"` + ApplyPoolSize *int64 `json:"apply-pool-size,omitempty" toml:"apply-pool-size,omitempty"` // +optional - StoreMaxBatchSize *int64 `json:"store-max-batch-size,omitempty",toml:"store-max-batch-size,omitempty"` + StoreMaxBatchSize *int64 `json:"store-max-batch-size,omitempty" toml:"store-max-batch-size,omitempty"` // +optional - StorePoolSize *int64 `json:"store-pool-size,omitempty",toml:"store-pool-size,omitempty"` + StorePoolSize *int64 `json:"store-pool-size,omitempty" toml:"store-pool-size,omitempty"` // +optional - HibernateRegions *bool `json:"hibernate-regions,omitempty",toml:"hibernate-regions,omitempty"` + HibernateRegions *bool `json:"hibernate-regions,omitempty" toml:"hibernate-regions,omitempty"` } diff --git a/pkg/apis/pingcap/v1alpha1/types.go b/pkg/apis/pingcap/v1alpha1/types.go index 5f77227ae3..07f950894b 100644 --- a/pkg/apis/pingcap/v1alpha1/types.go +++ b/pkg/apis/pingcap/v1alpha1/types.go @@ -182,7 +182,7 @@ type PDSpec struct { Service *ServiceSpec `json:"service,omitempty"` StorageClassName string `json:"storageClassName,omitempty"` - // Config is the Configuration of tidb-servers + // Config is the Configuration of pd-servers Config *PDConfig `json:"config,omitempty"` } @@ -200,9 +200,8 @@ type TiKVSpec struct { StorageClassName string `json:"storageClassName,omitempty"` MaxFailoverCount int32 `json:"maxFailoverCount,omitempty"` - // +k8s:openapi-gen=false - // TODO: add schema - config.GenericConfig `json:",inline"` + // Config is the Configuration of tikv-servers + Config *TiKVConfig `json:"config,omitempty"` } // +k8s:openapi-gen=true diff --git a/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go index 3ad853cd03..f1d46f405c 100644 --- a/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go +++ b/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go @@ -1926,6 +1926,183 @@ func (in *TiDBStatus) DeepCopy() *TiDBStatus { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TiKVBlockCacheConfig) DeepCopyInto(out *TiKVBlockCacheConfig) { + *out = *in + if in.Shared != nil { + in, out := &in.Shared, &out.Shared + *out = new(bool) + **out = **in + } + if in.NumShardBits != nil { + in, out := &in.NumShardBits, &out.NumShardBits + *out = new(int64) + **out = **in + } + if in.StrictCapacityLimit != nil { + in, out := &in.StrictCapacityLimit, &out.StrictCapacityLimit + *out = new(bool) + **out = **in + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TiKVBlockCacheConfig. +func (in *TiKVBlockCacheConfig) DeepCopy() *TiKVBlockCacheConfig { + if in == nil { + return nil + } + out := new(TiKVBlockCacheConfig) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TiKVCfConfig) DeepCopyInto(out *TiKVCfConfig) { + *out = *in + if in.DisableBlockCache != nil { + in, out := &in.DisableBlockCache, &out.DisableBlockCache + *out = new(bool) + **out = **in + } + if in.CacheIndexAndFilterBlocks != nil { + in, out := &in.CacheIndexAndFilterBlocks, &out.CacheIndexAndFilterBlocks + *out = new(bool) + **out = **in + } + if in.PinL0FilterAndIndexBlocks != nil { + in, out := &in.PinL0FilterAndIndexBlocks, &out.PinL0FilterAndIndexBlocks + *out = new(bool) + **out = **in + } + if in.UseBloomFilter != nil { + in, out := &in.UseBloomFilter, &out.UseBloomFilter + *out = new(bool) + **out = **in + } + if in.OptimizeFiltersForHits != nil { + in, out := &in.OptimizeFiltersForHits, &out.OptimizeFiltersForHits + *out = new(bool) + **out = **in + } + if in.WholeKeyFiltering != nil { + in, out := &in.WholeKeyFiltering, &out.WholeKeyFiltering + *out = new(bool) + **out = **in + } + if in.BloomFilterBitsPerKey != nil { + in, out := &in.BloomFilterBitsPerKey, &out.BloomFilterBitsPerKey + *out = new(int64) + **out = **in + } + if in.BlockBasedBloomFilter != nil { + in, out := &in.BlockBasedBloomFilter, &out.BlockBasedBloomFilter + *out = new(bool) + **out = **in + } + if in.ReadAmpBytesPerBit != nil { + in, out := &in.ReadAmpBytesPerBit, &out.ReadAmpBytesPerBit + *out = new(int64) + **out = **in + } + if in.CompressionPerLevel != nil { + in, out := &in.CompressionPerLevel, &out.CompressionPerLevel + *out = make([]string, len(*in)) + copy(*out, *in) + } + if in.MaxWriteBufferNumber != nil { + in, out := &in.MaxWriteBufferNumber, &out.MaxWriteBufferNumber + *out = new(int64) + **out = **in + } + if in.MinWriteBufferNumberToMerge != nil { + in, out := &in.MinWriteBufferNumberToMerge, &out.MinWriteBufferNumberToMerge + *out = new(int64) + **out = **in + } + if in.Level0FileNumCompactionTrigger != nil { + in, out := &in.Level0FileNumCompactionTrigger, &out.Level0FileNumCompactionTrigger + *out = new(int64) + **out = **in + } + if in.Level0SlowdownWritesTrigger != nil { + in, out := &in.Level0SlowdownWritesTrigger, &out.Level0SlowdownWritesTrigger + *out = new(int64) + **out = **in + } + if in.Level0StopWritesTrigger != nil { + in, out := &in.Level0StopWritesTrigger, &out.Level0StopWritesTrigger + *out = new(int64) + **out = **in + } + if in.CompactionPri != nil { + in, out := &in.CompactionPri, &out.CompactionPri + *out = new(int64) + **out = **in + } + if in.DynamicLevelBytes != nil { + in, out := &in.DynamicLevelBytes, &out.DynamicLevelBytes + *out = new(bool) + **out = **in + } + if in.NumLevels != nil { + in, out := &in.NumLevels, &out.NumLevels + *out = new(int64) + **out = **in + } + if in.MaxBytesForLevelMultiplier != nil { + in, out := &in.MaxBytesForLevelMultiplier, &out.MaxBytesForLevelMultiplier + *out = new(int64) + **out = **in + } + if in.CompactionStyle != nil { + in, out := &in.CompactionStyle, &out.CompactionStyle + *out = new(int64) + **out = **in + } + if in.DisableAutoCompactions != nil { + in, out := &in.DisableAutoCompactions, &out.DisableAutoCompactions + *out = new(bool) + **out = **in + } + if in.ForceConsistencyChecks != nil { + in, out := &in.ForceConsistencyChecks, &out.ForceConsistencyChecks + *out = new(bool) + **out = **in + } + if in.PropSizeIndexDistance != nil { + in, out := &in.PropSizeIndexDistance, &out.PropSizeIndexDistance + *out = new(int64) + **out = **in + } + if in.PropKeysIndexDistance != nil { + in, out := &in.PropKeysIndexDistance, &out.PropKeysIndexDistance + *out = new(int64) + **out = **in + } + if in.EnableDoublySkiplist != nil { + in, out := &in.EnableDoublySkiplist, &out.EnableDoublySkiplist + *out = new(bool) + **out = **in + } + if in.Titan != nil { + in, out := &in.Titan, &out.Titan + *out = new(TiKVTitanCfConfig) + **out = **in + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TiKVCfConfig. +func (in *TiKVCfConfig) DeepCopy() *TiKVCfConfig { + if in == nil { + return nil + } + out := new(TiKVCfConfig) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *TiKVClient) DeepCopyInto(out *TiKVClient) { *out = *in @@ -1982,6 +2159,215 @@ func (in *TiKVClient) DeepCopy() *TiKVClient { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TiKVConfig) DeepCopyInto(out *TiKVConfig) { + *out = *in + if in.PanicWhenUnexpectedKeyOrData != nil { + in, out := &in.PanicWhenUnexpectedKeyOrData, &out.PanicWhenUnexpectedKeyOrData + *out = new(bool) + **out = **in + } + if in.GrpcConcurrency != nil { + in, out := &in.GrpcConcurrency, &out.GrpcConcurrency + *out = new(uint) + **out = **in + } + if in.GrpcConcurrentStream != nil { + in, out := &in.GrpcConcurrentStream, &out.GrpcConcurrentStream + *out = new(uint) + **out = **in + } + if in.GrpcRaftConnNum != nil { + in, out := &in.GrpcRaftConnNum, &out.GrpcRaftConnNum + *out = new(uint) + **out = **in + } + if in.ConcurrentSendSnapLimit != nil { + in, out := &in.ConcurrentSendSnapLimit, &out.ConcurrentSendSnapLimit + *out = new(uint) + **out = **in + } + if in.ConcurrentRecvSnapLimit != nil { + in, out := &in.ConcurrentRecvSnapLimit, &out.ConcurrentRecvSnapLimit + *out = new(uint) + **out = **in + } + if in.EndPointRecursionLimit != nil { + in, out := &in.EndPointRecursionLimit, &out.EndPointRecursionLimit + *out = new(uint) + **out = **in + } + if in.EndPointStreamChannelSize != nil { + in, out := &in.EndPointStreamChannelSize, &out.EndPointStreamChannelSize + *out = new(uint) + **out = **in + } + if in.EndPointBatchRowLimit != nil { + in, out := &in.EndPointBatchRowLimit, &out.EndPointBatchRowLimit + *out = new(uint) + **out = **in + } + if in.EndPointStreamBatchRowLimit != nil { + in, out := &in.EndPointStreamBatchRowLimit, &out.EndPointStreamBatchRowLimit + *out = new(uint) + **out = **in + } + if in.EndPointEnableBatchIfPossible != nil { + in, out := &in.EndPointEnableBatchIfPossible, &out.EndPointEnableBatchIfPossible + *out = new(uint) + **out = **in + } + if in.StatsConcurrency != nil { + in, out := &in.StatsConcurrency, &out.StatsConcurrency + *out = new(uint) + **out = **in + } + if in.HeavyLoadThreshold != nil { + in, out := &in.HeavyLoadThreshold, &out.HeavyLoadThreshold + *out = new(uint) + **out = **in + } + if in.Labels != nil { + in, out := &in.Labels, &out.Labels + *out = make(map[string]string, len(*in)) + for key, val := range *in { + (*out)[key] = val + } + } + if in.Server != nil { + in, out := &in.Server, &out.Server + *out = new(TiKVServerConfig) + (*in).DeepCopyInto(*out) + } + if in.Storage != nil { + in, out := &in.Storage, &out.Storage + *out = new(TiKVStorageConfig) + (*in).DeepCopyInto(*out) + } + if in.Raftstore != nil { + in, out := &in.Raftstore, &out.Raftstore + *out = new(TiKVRaftstoreConfig) + (*in).DeepCopyInto(*out) + } + if in.Rocksdb != nil { + in, out := &in.Rocksdb, &out.Rocksdb + *out = new(TiKVDbConfig) + (*in).DeepCopyInto(*out) + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TiKVConfig. +func (in *TiKVConfig) DeepCopy() *TiKVConfig { + if in == nil { + return nil + } + out := new(TiKVConfig) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TiKVDbConfig) DeepCopyInto(out *TiKVDbConfig) { + *out = *in + if in.WalRecoveryMode != nil { + in, out := &in.WalRecoveryMode, &out.WalRecoveryMode + *out = new(int64) + **out = **in + } + if in.WalTTLSeconds != nil { + in, out := &in.WalTTLSeconds, &out.WalTTLSeconds + *out = new(int64) + **out = **in + } + if in.MaxBackgroundJobs != nil { + in, out := &in.MaxBackgroundJobs, &out.MaxBackgroundJobs + *out = new(int64) + **out = **in + } + if in.CreateIfMissing != nil { + in, out := &in.CreateIfMissing, &out.CreateIfMissing + *out = new(bool) + **out = **in + } + if in.MaxOpenFiles != nil { + in, out := &in.MaxOpenFiles, &out.MaxOpenFiles + *out = new(int64) + **out = **in + } + if in.EnableStatistics != nil { + in, out := &in.EnableStatistics, &out.EnableStatistics + *out = new(bool) + **out = **in + } + if in.InfoLogKeepLogFileNum != nil { + in, out := &in.InfoLogKeepLogFileNum, &out.InfoLogKeepLogFileNum + *out = new(int64) + **out = **in + } + if in.RateLimiterMode != nil { + in, out := &in.RateLimiterMode, &out.RateLimiterMode + *out = new(int64) + **out = **in + } + if in.AutoTuned != nil { + in, out := &in.AutoTuned, &out.AutoTuned + *out = new(bool) + **out = **in + } + if in.MaxSubCompactions != nil { + in, out := &in.MaxSubCompactions, &out.MaxSubCompactions + *out = new(int64) + **out = **in + } + if in.UseDirectIoForFlushAndCompaction != nil { + in, out := &in.UseDirectIoForFlushAndCompaction, &out.UseDirectIoForFlushAndCompaction + *out = new(bool) + **out = **in + } + if in.EnablePipelinedWrite != nil { + in, out := &in.EnablePipelinedWrite, &out.EnablePipelinedWrite + *out = new(bool) + **out = **in + } + if in.Defaultcf != nil { + in, out := &in.Defaultcf, &out.Defaultcf + *out = new(TiKVCfConfig) + (*in).DeepCopyInto(*out) + } + if in.Writecf != nil { + in, out := &in.Writecf, &out.Writecf + *out = new(TiKVCfConfig) + (*in).DeepCopyInto(*out) + } + if in.Lockcf != nil { + in, out := &in.Lockcf, &out.Lockcf + *out = new(TiKVCfConfig) + (*in).DeepCopyInto(*out) + } + if in.Raftcf != nil { + in, out := &in.Raftcf, &out.Raftcf + *out = new(TiKVCfConfig) + (*in).DeepCopyInto(*out) + } + if in.Titan != nil { + in, out := &in.Titan, &out.Titan + *out = new(TiKVTitanDBConfig) + (*in).DeepCopyInto(*out) + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TiKVDbConfig. +func (in *TiKVDbConfig) DeepCopy() *TiKVDbConfig { + if in == nil { + return nil + } + out := new(TiKVDbConfig) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *TiKVFailureStore) DeepCopyInto(out *TiKVFailureStore) { *out = *in @@ -1999,6 +2385,205 @@ func (in *TiKVFailureStore) DeepCopy() *TiKVFailureStore { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TiKVRaftstoreConfig) DeepCopyInto(out *TiKVRaftstoreConfig) { + *out = *in + if in.SyncLog != nil { + in, out := &in.SyncLog, &out.SyncLog + *out = new(bool) + **out = **in + } + if in.RaftHeartbeatTicks != nil { + in, out := &in.RaftHeartbeatTicks, &out.RaftHeartbeatTicks + *out = new(int64) + **out = **in + } + if in.RaftElectionTimeoutTicks != nil { + in, out := &in.RaftElectionTimeoutTicks, &out.RaftElectionTimeoutTicks + *out = new(int64) + **out = **in + } + if in.RaftLogGCThreshold != nil { + in, out := &in.RaftLogGCThreshold, &out.RaftLogGCThreshold + *out = new(int64) + **out = **in + } + if in.RaftLogGCCountLimit != nil { + in, out := &in.RaftLogGCCountLimit, &out.RaftLogGCCountLimit + *out = new(int64) + **out = **in + } + if in.RegionCompactCheckStep != nil { + in, out := &in.RegionCompactCheckStep, &out.RegionCompactCheckStep + *out = new(int64) + **out = **in + } + if in.RegionCompactMinTombstones != nil { + in, out := &in.RegionCompactMinTombstones, &out.RegionCompactMinTombstones + *out = new(int64) + **out = **in + } + if in.RegionCompactTombstonesPercent != nil { + in, out := &in.RegionCompactTombstonesPercent, &out.RegionCompactTombstonesPercent + *out = new(int64) + **out = **in + } + if in.NotifyCapacity != nil { + in, out := &in.NotifyCapacity, &out.NotifyCapacity + *out = new(int64) + **out = **in + } + if in.MessagesPerTick != nil { + in, out := &in.MessagesPerTick, &out.MessagesPerTick + *out = new(int64) + **out = **in + } + if in.LeaderTransferMaxLogLag != nil { + in, out := &in.LeaderTransferMaxLogLag, &out.LeaderTransferMaxLogLag + *out = new(int64) + **out = **in + } + if in.RightDeriveWhenSplit != nil { + in, out := &in.RightDeriveWhenSplit, &out.RightDeriveWhenSplit + *out = new(bool) + **out = **in + } + if in.AllowRemoveLeader != nil { + in, out := &in.AllowRemoveLeader, &out.AllowRemoveLeader + *out = new(bool) + **out = **in + } + if in.MergeMaxLogGap != nil { + in, out := &in.MergeMaxLogGap, &out.MergeMaxLogGap + *out = new(int64) + **out = **in + } + if in.UseDeleteRange != nil { + in, out := &in.UseDeleteRange, &out.UseDeleteRange + *out = new(bool) + **out = **in + } + if in.ApplyMaxBatchSize != nil { + in, out := &in.ApplyMaxBatchSize, &out.ApplyMaxBatchSize + *out = new(int64) + **out = **in + } + if in.ApplyPoolSize != nil { + in, out := &in.ApplyPoolSize, &out.ApplyPoolSize + *out = new(int64) + **out = **in + } + if in.StoreMaxBatchSize != nil { + in, out := &in.StoreMaxBatchSize, &out.StoreMaxBatchSize + *out = new(int64) + **out = **in + } + if in.StorePoolSize != nil { + in, out := &in.StorePoolSize, &out.StorePoolSize + *out = new(int64) + **out = **in + } + if in.HibernateRegions != nil { + in, out := &in.HibernateRegions, &out.HibernateRegions + *out = new(bool) + **out = **in + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TiKVRaftstoreConfig. +func (in *TiKVRaftstoreConfig) DeepCopy() *TiKVRaftstoreConfig { + if in == nil { + return nil + } + out := new(TiKVRaftstoreConfig) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TiKVServerConfig) DeepCopyInto(out *TiKVServerConfig) { + *out = *in + if in.GrpcConcurrency != nil { + in, out := &in.GrpcConcurrency, &out.GrpcConcurrency + *out = new(int64) + **out = **in + } + if in.GrpcConcurrentStream != nil { + in, out := &in.GrpcConcurrentStream, &out.GrpcConcurrentStream + *out = new(int64) + **out = **in + } + if in.GrpcRaftConnNum != nil { + in, out := &in.GrpcRaftConnNum, &out.GrpcRaftConnNum + *out = new(int64) + **out = **in + } + if in.ConcurrentSendSnapLimit != nil { + in, out := &in.ConcurrentSendSnapLimit, &out.ConcurrentSendSnapLimit + *out = new(int64) + **out = **in + } + if in.ConcurrentRecvSnapLimit != nil { + in, out := &in.ConcurrentRecvSnapLimit, &out.ConcurrentRecvSnapLimit + *out = new(int64) + **out = **in + } + if in.EndPointRecursionLimit != nil { + in, out := &in.EndPointRecursionLimit, &out.EndPointRecursionLimit + *out = new(int64) + **out = **in + } + if in.EndPointStreamChannelSize != nil { + in, out := &in.EndPointStreamChannelSize, &out.EndPointStreamChannelSize + *out = new(int64) + **out = **in + } + if in.EndPointBatchRowLimit != nil { + in, out := &in.EndPointBatchRowLimit, &out.EndPointBatchRowLimit + *out = new(int64) + **out = **in + } + if in.EndPointStreamBatchRowLimit != nil { + in, out := &in.EndPointStreamBatchRowLimit, &out.EndPointStreamBatchRowLimit + *out = new(int64) + **out = **in + } + if in.EndPointEnableBatchIfPossible != nil { + in, out := &in.EndPointEnableBatchIfPossible, &out.EndPointEnableBatchIfPossible + *out = new(bool) + **out = **in + } + if in.StatsConcurrency != nil { + in, out := &in.StatsConcurrency, &out.StatsConcurrency + *out = new(int64) + **out = **in + } + if in.HeavyLoadThreshold != nil { + in, out := &in.HeavyLoadThreshold, &out.HeavyLoadThreshold + *out = new(int64) + **out = **in + } + if in.Labels != nil { + in, out := &in.Labels, &out.Labels + *out = make(map[string]string, len(*in)) + for key, val := range *in { + (*out)[key] = val + } + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TiKVServerConfig. +func (in *TiKVServerConfig) DeepCopy() *TiKVServerConfig { + if in == nil { + return nil + } + out := new(TiKVServerConfig) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *TiKVSpec) DeepCopyInto(out *TiKVSpec) { *out = *in @@ -2009,7 +2594,11 @@ func (in *TiKVSpec) DeepCopyInto(out *TiKVSpec) { *out = new(ServiceSpec) (*in).DeepCopyInto(*out) } - in.GenericConfig.DeepCopyInto(&out.GenericConfig) + if in.Config != nil { + in, out := &in.Config, &out.Config + *out = new(TiKVConfig) + (*in).DeepCopyInto(*out) + } return } @@ -2065,6 +2654,47 @@ func (in *TiKVStatus) DeepCopy() *TiKVStatus { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TiKVStorageConfig) DeepCopyInto(out *TiKVStorageConfig) { + *out = *in + if in.MaxKeySize != nil { + in, out := &in.MaxKeySize, &out.MaxKeySize + *out = new(int64) + **out = **in + } + if in.SchedulerNotifyCapacity != nil { + in, out := &in.SchedulerNotifyCapacity, &out.SchedulerNotifyCapacity + *out = new(int64) + **out = **in + } + if in.SchedulerConcurrency != nil { + in, out := &in.SchedulerConcurrency, &out.SchedulerConcurrency + *out = new(int64) + **out = **in + } + if in.SchedulerWorkerPoolSize != nil { + in, out := &in.SchedulerWorkerPoolSize, &out.SchedulerWorkerPoolSize + *out = new(int64) + **out = **in + } + if in.BlockCache != nil { + in, out := &in.BlockCache, &out.BlockCache + *out = new(TiKVBlockCacheConfig) + (*in).DeepCopyInto(*out) + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TiKVStorageConfig. +func (in *TiKVStorageConfig) DeepCopy() *TiKVStorageConfig { + if in == nil { + return nil + } + out := new(TiKVStorageConfig) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *TiKVStore) DeepCopyInto(out *TiKVStore) { *out = *in @@ -2083,6 +2713,53 @@ func (in *TiKVStore) DeepCopy() *TiKVStore { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TiKVTitanCfConfig) DeepCopyInto(out *TiKVTitanCfConfig) { + *out = *in + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TiKVTitanCfConfig. +func (in *TiKVTitanCfConfig) DeepCopy() *TiKVTitanCfConfig { + if in == nil { + return nil + } + out := new(TiKVTitanCfConfig) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TiKVTitanDBConfig) DeepCopyInto(out *TiKVTitanDBConfig) { + *out = *in + if in.Enabled != nil { + in, out := &in.Enabled, &out.Enabled + *out = new(bool) + **out = **in + } + if in.DisableGc != nil { + in, out := &in.DisableGc, &out.DisableGc + *out = new(bool) + **out = **in + } + if in.MaxBackgroundGc != nil { + in, out := &in.MaxBackgroundGc, &out.MaxBackgroundGc + *out = new(int64) + **out = **in + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TiKVTitanDBConfig. +func (in *TiKVTitanDBConfig) DeepCopy() *TiKVTitanDBConfig { + if in == nil { + return nil + } + out := new(TiKVTitanDBConfig) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *TidbCluster) DeepCopyInto(out *TidbCluster) { *out = *in