master.go 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329
  1. package command
  2. import (
  3. "context"
  4. "fmt"
  5. "net/http"
  6. "os"
  7. "path"
  8. "strings"
  9. "time"
  10. hashicorpRaft "github.com/hashicorp/raft"
  11. "golang.org/x/exp/slices"
  12. "github.com/gorilla/mux"
  13. "github.com/seaweedfs/raft/protobuf"
  14. "github.com/spf13/viper"
  15. "google.golang.org/grpc/reflection"
  16. stats_collect "github.com/seaweedfs/seaweedfs/weed/stats"
  17. "github.com/seaweedfs/seaweedfs/weed/util/grace"
  18. "github.com/seaweedfs/seaweedfs/weed/glog"
  19. "github.com/seaweedfs/seaweedfs/weed/pb"
  20. "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
  21. "github.com/seaweedfs/seaweedfs/weed/security"
  22. weed_server "github.com/seaweedfs/seaweedfs/weed/server"
  23. "github.com/seaweedfs/seaweedfs/weed/storage/backend"
  24. "github.com/seaweedfs/seaweedfs/weed/util"
  25. )
  26. var (
  27. m MasterOptions
  28. )
  29. type MasterOptions struct {
  30. port *int
  31. portGrpc *int
  32. ip *string
  33. ipBind *string
  34. metaFolder *string
  35. peers *string
  36. volumeSizeLimitMB *uint
  37. volumePreallocate *bool
  38. maxParallelVacuumPerServer *int
  39. // pulseSeconds *int
  40. defaultReplication *string
  41. garbageThreshold *float64
  42. whiteList *string
  43. disableHttp *bool
  44. metricsAddress *string
  45. metricsIntervalSec *int
  46. raftResumeState *bool
  47. metricsHttpPort *int
  48. metricsHttpIp *string
  49. heartbeatInterval *time.Duration
  50. electionTimeout *time.Duration
  51. raftHashicorp *bool
  52. raftBootstrap *bool
  53. }
  54. func init() {
  55. cmdMaster.Run = runMaster // break init cycle
  56. m.port = cmdMaster.Flag.Int("port", 9333, "http listen port")
  57. m.portGrpc = cmdMaster.Flag.Int("port.grpc", 0, "grpc listen port")
  58. m.ip = cmdMaster.Flag.String("ip", util.DetectedHostAddress(), "master <ip>|<server> address, also used as identifier")
  59. m.ipBind = cmdMaster.Flag.String("ip.bind", "", "ip address to bind to. If empty, default to same as -ip option.")
  60. m.metaFolder = cmdMaster.Flag.String("mdir", os.TempDir(), "data directory to store meta data")
  61. m.peers = cmdMaster.Flag.String("peers", "", "all master nodes in comma separated ip:port list, example: 127.0.0.1:9093,127.0.0.1:9094,127.0.0.1:9095")
  62. m.volumeSizeLimitMB = cmdMaster.Flag.Uint("volumeSizeLimitMB", 30*1000, "Master stops directing writes to oversized volumes.")
  63. m.volumePreallocate = cmdMaster.Flag.Bool("volumePreallocate", false, "Preallocate disk space for volumes.")
  64. m.maxParallelVacuumPerServer = cmdMaster.Flag.Int("maxParallelVacuumPerServer", 1, "maximum number of volumes to vacuum in parallel per volume server")
  65. // m.pulseSeconds = cmdMaster.Flag.Int("pulseSeconds", 5, "number of seconds between heartbeats")
  66. m.defaultReplication = cmdMaster.Flag.String("defaultReplication", "", "Default replication type if not specified.")
  67. m.garbageThreshold = cmdMaster.Flag.Float64("garbageThreshold", 0.3, "threshold to vacuum and reclaim spaces")
  68. m.whiteList = cmdMaster.Flag.String("whiteList", "", "comma separated Ip addresses having write permission. No limit if empty.")
  69. m.disableHttp = cmdMaster.Flag.Bool("disableHttp", false, "disable http requests, only gRPC operations are allowed.")
  70. m.metricsAddress = cmdMaster.Flag.String("metrics.address", "", "Prometheus gateway address <host>:<port>")
  71. m.metricsIntervalSec = cmdMaster.Flag.Int("metrics.intervalSeconds", 15, "Prometheus push interval in seconds")
  72. m.metricsHttpPort = cmdMaster.Flag.Int("metricsPort", 0, "Prometheus metrics listen port")
  73. m.metricsHttpIp = cmdMaster.Flag.String("metricsIp", "", "metrics listen ip. If empty, default to same as -ip.bind option.")
  74. m.raftResumeState = cmdMaster.Flag.Bool("resumeState", false, "resume previous state on start master server")
  75. m.heartbeatInterval = cmdMaster.Flag.Duration("heartbeatInterval", 300*time.Millisecond, "heartbeat interval of master servers, and will be randomly multiplied by [1, 1.25)")
  76. m.electionTimeout = cmdMaster.Flag.Duration("electionTimeout", 10*time.Second, "election timeout of master servers")
  77. m.raftHashicorp = cmdMaster.Flag.Bool("raftHashicorp", false, "use hashicorp raft")
  78. m.raftBootstrap = cmdMaster.Flag.Bool("raftBootstrap", false, "Whether to bootstrap the Raft cluster")
  79. }
  80. var cmdMaster = &Command{
  81. UsageLine: "master -port=9333",
  82. Short: "start a master server",
  83. Long: `start a master server to provide volume=>location mapping service and sequence number of file ids
  84. The configuration file "security.toml" is read from ".", "$HOME/.seaweedfs/", "/usr/local/etc/seaweedfs/", or "/etc/seaweedfs/", in that order.
  85. The example security.toml configuration file can be generated by "weed scaffold -config=security"
  86. `,
  87. }
  88. var (
  89. masterCpuProfile = cmdMaster.Flag.String("cpuprofile", "", "cpu profile output file")
  90. masterMemProfile = cmdMaster.Flag.String("memprofile", "", "memory profile output file")
  91. )
  92. func runMaster(cmd *Command, args []string) bool {
  93. util.LoadSecurityConfiguration()
  94. util.LoadConfiguration("master", false)
  95. grace.SetupProfiling(*masterCpuProfile, *masterMemProfile)
  96. parent, _ := util.FullPath(*m.metaFolder).DirAndName()
  97. if util.FileExists(string(parent)) && !util.FileExists(*m.metaFolder) {
  98. os.MkdirAll(*m.metaFolder, 0755)
  99. }
  100. if err := util.TestFolderWritable(util.ResolvePath(*m.metaFolder)); err != nil {
  101. glog.Fatalf("Check Meta Folder (-mdir) Writable %s : %s", *m.metaFolder, err)
  102. }
  103. masterWhiteList := util.StringSplit(*m.whiteList, ",")
  104. if *m.volumeSizeLimitMB > util.VolumeSizeLimitGB*1000 {
  105. glog.Fatalf("volumeSizeLimitMB should be smaller than 30000")
  106. }
  107. switch {
  108. case *m.metricsHttpIp != "":
  109. // noting to do, use m.metricsHttpIp
  110. case *m.ipBind != "":
  111. *m.metricsHttpIp = *m.ipBind
  112. case *m.ip != "":
  113. *m.metricsHttpIp = *m.ip
  114. }
  115. go stats_collect.StartMetricsServer(*m.metricsHttpIp, *m.metricsHttpPort)
  116. go stats_collect.LoopPushingMetric("masterServer", util.JoinHostPort(*m.ip, *m.port), *m.metricsAddress, *m.metricsIntervalSec)
  117. startMaster(m, masterWhiteList)
  118. return true
  119. }
  120. func startMaster(masterOption MasterOptions, masterWhiteList []string) {
  121. backend.LoadConfiguration(util.GetViper())
  122. if *masterOption.portGrpc == 0 {
  123. *masterOption.portGrpc = 10000 + *masterOption.port
  124. }
  125. if *masterOption.ipBind == "" {
  126. *masterOption.ipBind = *masterOption.ip
  127. }
  128. myMasterAddress, peers := checkPeers(*masterOption.ip, *masterOption.port, *masterOption.portGrpc, *masterOption.peers)
  129. masterPeers := make(map[string]pb.ServerAddress)
  130. for _, peer := range peers {
  131. masterPeers[string(peer)] = peer
  132. }
  133. r := mux.NewRouter()
  134. ms := weed_server.NewMasterServer(r, masterOption.toMasterOption(masterWhiteList), masterPeers)
  135. listeningAddress := util.JoinHostPort(*masterOption.ipBind, *masterOption.port)
  136. glog.V(0).Infof("Start Seaweed Master %s at %s", util.Version(), listeningAddress)
  137. masterListener, masterLocalListener, e := util.NewIpAndLocalListeners(*masterOption.ipBind, *masterOption.port, 0)
  138. if e != nil {
  139. glog.Fatalf("Master startup error: %v", e)
  140. }
  141. // start raftServer
  142. metaDir := path.Join(*masterOption.metaFolder, fmt.Sprintf("m%d", *masterOption.port))
  143. raftServerOption := &weed_server.RaftServerOption{
  144. GrpcDialOption: security.LoadClientTLS(util.GetViper(), "grpc.master"),
  145. Peers: masterPeers,
  146. ServerAddr: myMasterAddress,
  147. DataDir: util.ResolvePath(metaDir),
  148. Topo: ms.Topo,
  149. RaftResumeState: *masterOption.raftResumeState,
  150. HeartbeatInterval: *masterOption.heartbeatInterval,
  151. ElectionTimeout: *masterOption.electionTimeout,
  152. RaftBootstrap: *masterOption.raftBootstrap,
  153. }
  154. var raftServer *weed_server.RaftServer
  155. var err error
  156. if *masterOption.raftHashicorp {
  157. if raftServer, err = weed_server.NewHashicorpRaftServer(raftServerOption); err != nil {
  158. glog.Fatalf("NewHashicorpRaftServer: %s", err)
  159. }
  160. } else {
  161. raftServer, err = weed_server.NewRaftServer(raftServerOption)
  162. if raftServer == nil {
  163. glog.Fatalf("please verify %s is writable, see https://github.com/seaweedfs/seaweedfs/issues/717: %s", *masterOption.metaFolder, err)
  164. }
  165. }
  166. ms.SetRaftServer(raftServer)
  167. r.HandleFunc("/cluster/status", raftServer.StatusHandler).Methods(http.MethodGet)
  168. r.HandleFunc("/cluster/healthz", raftServer.HealthzHandler).Methods(http.MethodGet, http.MethodHead)
  169. if *masterOption.raftHashicorp {
  170. r.HandleFunc("/raft/stats", raftServer.StatsRaftHandler).Methods(http.MethodGet)
  171. }
  172. // starting grpc server
  173. grpcPort := *masterOption.portGrpc
  174. grpcL, grpcLocalL, err := util.NewIpAndLocalListeners(*masterOption.ipBind, grpcPort, 0)
  175. if err != nil {
  176. glog.Fatalf("master failed to listen on grpc port %d: %v", grpcPort, err)
  177. }
  178. grpcS := pb.NewGrpcServer(security.LoadServerTLS(util.GetViper(), "grpc.master"))
  179. master_pb.RegisterSeaweedServer(grpcS, ms)
  180. if *masterOption.raftHashicorp {
  181. raftServer.TransportManager.Register(grpcS)
  182. } else {
  183. protobuf.RegisterRaftServer(grpcS, raftServer)
  184. }
  185. reflection.Register(grpcS)
  186. glog.V(0).Infof("Start Seaweed Master %s grpc server at %s:%d", util.Version(), *masterOption.ipBind, grpcPort)
  187. if grpcLocalL != nil {
  188. go grpcS.Serve(grpcLocalL)
  189. }
  190. go grpcS.Serve(grpcL)
  191. timeSleep := 1500 * time.Millisecond
  192. if !*masterOption.raftHashicorp {
  193. go func() {
  194. time.Sleep(timeSleep)
  195. ms.Topo.RaftServerAccessLock.RLock()
  196. isEmptyMaster := ms.Topo.RaftServer.Leader() == "" && ms.Topo.RaftServer.IsLogEmpty()
  197. if isEmptyMaster && isTheFirstOne(myMasterAddress, peers) && ms.MasterClient.FindLeaderFromOtherPeers(myMasterAddress) == "" {
  198. raftServer.DoJoinCommand()
  199. }
  200. ms.Topo.RaftServerAccessLock.RUnlock()
  201. }()
  202. }
  203. go ms.MasterClient.KeepConnectedToMaster(context.Background())
  204. // start http server
  205. var (
  206. clientCertFile,
  207. certFile,
  208. keyFile string
  209. )
  210. useTLS := false
  211. useMTLS := false
  212. if viper.GetString("https.master.key") != "" {
  213. useTLS = true
  214. certFile = viper.GetString("https.master.cert")
  215. keyFile = viper.GetString("https.master.key")
  216. }
  217. if viper.GetString("https.master.ca") != "" {
  218. useMTLS = true
  219. clientCertFile = viper.GetString("https.master.ca")
  220. }
  221. httpS := &http.Server{Handler: r}
  222. if masterLocalListener != nil {
  223. go httpS.Serve(masterLocalListener)
  224. }
  225. if useMTLS {
  226. httpS.TLSConfig = security.LoadClientTLSHTTP(clientCertFile)
  227. }
  228. if useTLS {
  229. go httpS.ServeTLS(masterListener, certFile, keyFile)
  230. } else {
  231. go httpS.Serve(masterListener)
  232. }
  233. grace.OnInterrupt(ms.Shutdown)
  234. grace.OnInterrupt(grpcS.Stop)
  235. grace.OnReload(func() {
  236. if ms.Topo.HashicorpRaft != nil && ms.Topo.HashicorpRaft.State() == hashicorpRaft.Leader {
  237. ms.Topo.HashicorpRaft.LeadershipTransfer()
  238. }
  239. })
  240. select {}
  241. }
  242. func checkPeers(masterIp string, masterPort int, masterGrpcPort int, peers string) (masterAddress pb.ServerAddress, cleanedPeers []pb.ServerAddress) {
  243. glog.V(0).Infof("current: %s:%d peers:%s", masterIp, masterPort, peers)
  244. masterAddress = pb.NewServerAddress(masterIp, masterPort, masterGrpcPort)
  245. cleanedPeers = pb.ServerAddresses(peers).ToAddresses()
  246. hasSelf := false
  247. for _, peer := range cleanedPeers {
  248. if peer.ToHttpAddress() == masterAddress.ToHttpAddress() {
  249. hasSelf = true
  250. break
  251. }
  252. }
  253. if !hasSelf {
  254. cleanedPeers = append(cleanedPeers, masterAddress)
  255. }
  256. if len(cleanedPeers)%2 == 0 {
  257. glog.Fatalf("Only odd number of masters are supported: %+v", cleanedPeers)
  258. }
  259. return
  260. }
  261. func isTheFirstOne(self pb.ServerAddress, peers []pb.ServerAddress) bool {
  262. slices.SortFunc(peers, func(a, b pb.ServerAddress) int {
  263. return strings.Compare(string(a), string(b))
  264. })
  265. if len(peers) <= 0 {
  266. return true
  267. }
  268. return self == peers[0]
  269. }
  270. func (m *MasterOptions) toMasterOption(whiteList []string) *weed_server.MasterOption {
  271. masterAddress := pb.NewServerAddress(*m.ip, *m.port, *m.portGrpc)
  272. return &weed_server.MasterOption{
  273. Master: masterAddress,
  274. MetaFolder: *m.metaFolder,
  275. VolumeSizeLimitMB: uint32(*m.volumeSizeLimitMB),
  276. VolumePreallocate: *m.volumePreallocate,
  277. MaxParallelVacuumPerServer: *m.maxParallelVacuumPerServer,
  278. // PulseSeconds: *m.pulseSeconds,
  279. DefaultReplicaPlacement: *m.defaultReplication,
  280. GarbageThreshold: *m.garbageThreshold,
  281. WhiteList: whiteList,
  282. DisableHttp: *m.disableHttp,
  283. MetricsAddress: *m.metricsAddress,
  284. MetricsIntervalSec: *m.metricsIntervalSec,
  285. }
  286. }