You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

423 lines
14 KiB

6 years ago
6 years ago
6 years ago
6 years ago
8 months ago
5 years ago
3 years ago
6 years ago
8 months ago
6 years ago
6 years ago
3 years ago
3 years ago
3 years ago
3 years ago
3 years ago
3 years ago
2 years ago
3 years ago
  1. package weed_server
  2. import (
  3. "context"
  4. "fmt"
  5. "net/http"
  6. "net/http/httputil"
  7. "net/url"
  8. "os"
  9. "regexp"
  10. "strings"
  11. "sync"
  12. "time"
  13. "github.com/seaweedfs/seaweedfs/weed/stats"
  14. "github.com/seaweedfs/seaweedfs/weed/cluster"
  15. "github.com/seaweedfs/seaweedfs/weed/pb"
  16. "github.com/gorilla/mux"
  17. hashicorpRaft "github.com/hashicorp/raft"
  18. "github.com/seaweedfs/raft"
  19. "google.golang.org/grpc"
  20. "github.com/seaweedfs/seaweedfs/weed/glog"
  21. "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
  22. "github.com/seaweedfs/seaweedfs/weed/security"
  23. "github.com/seaweedfs/seaweedfs/weed/sequence"
  24. "github.com/seaweedfs/seaweedfs/weed/shell"
  25. "github.com/seaweedfs/seaweedfs/weed/topology"
  26. "github.com/seaweedfs/seaweedfs/weed/util"
  27. util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
  28. "github.com/seaweedfs/seaweedfs/weed/wdclient"
  29. )
  30. const (
  31. SequencerType = "master.sequencer.type"
  32. SequencerSnowflakeId = "master.sequencer.sequencer_snowflake_id"
  33. )
  34. type MasterOption struct {
  35. Master pb.ServerAddress
  36. MetaFolder string
  37. VolumeSizeLimitMB uint32
  38. VolumePreallocate bool
  39. MaxParallelVacuumPerServer int
  40. // PulseSeconds int
  41. DefaultReplicaPlacement string
  42. GarbageThreshold float64
  43. WhiteList []string
  44. DisableHttp bool
  45. MetricsAddress string
  46. MetricsIntervalSec int
  47. IsFollower bool
  48. }
  49. type MasterServer struct {
  50. master_pb.UnimplementedSeaweedServer
  51. option *MasterOption
  52. guard *security.Guard
  53. preallocateSize int64
  54. Topo *topology.Topology
  55. vg *topology.VolumeGrowth
  56. volumeGrowthRequestChan chan *topology.VolumeGrowRequest
  57. boundedLeaderChan chan int
  58. // notifying clients
  59. clientChansLock sync.RWMutex
  60. clientChans map[string]chan *master_pb.KeepConnectedResponse
  61. grpcDialOption grpc.DialOption
  62. MasterClient *wdclient.MasterClient
  63. adminLocks *AdminLocks
  64. Cluster *cluster.Cluster
  65. }
  66. func NewMasterServer(r *mux.Router, option *MasterOption, peers map[string]pb.ServerAddress) *MasterServer {
  67. v := util.GetViper()
  68. signingKey := v.GetString("jwt.signing.key")
  69. v.SetDefault("jwt.signing.expires_after_seconds", 10)
  70. expiresAfterSec := v.GetInt("jwt.signing.expires_after_seconds")
  71. readSigningKey := v.GetString("jwt.signing.read.key")
  72. v.SetDefault("jwt.signing.read.expires_after_seconds", 60)
  73. readExpiresAfterSec := v.GetInt("jwt.signing.read.expires_after_seconds")
  74. v.SetDefault("master.replication.treat_replication_as_minimums", false)
  75. replicationAsMin := v.GetBool("master.replication.treat_replication_as_minimums")
  76. v.SetDefault("master.volume_growth.copy_1", topology.VolumeGrowStrategy.Copy1Count)
  77. v.SetDefault("master.volume_growth.copy_2", topology.VolumeGrowStrategy.Copy2Count)
  78. v.SetDefault("master.volume_growth.copy_3", topology.VolumeGrowStrategy.Copy3Count)
  79. v.SetDefault("master.volume_growth.copy_other", topology.VolumeGrowStrategy.CopyOtherCount)
  80. v.SetDefault("master.volume_growth.threshold", topology.VolumeGrowStrategy.Threshold)
  81. topology.VolumeGrowStrategy.Copy1Count = v.GetUint32("master.volume_growth.copy_1")
  82. topology.VolumeGrowStrategy.Copy2Count = v.GetUint32("master.volume_growth.copy_2")
  83. topology.VolumeGrowStrategy.Copy3Count = v.GetUint32("master.volume_growth.copy_3")
  84. topology.VolumeGrowStrategy.CopyOtherCount = v.GetUint32("master.volume_growth.copy_other")
  85. topology.VolumeGrowStrategy.Threshold = v.GetFloat64("master.volume_growth.threshold")
  86. whiteList := util.StringSplit(v.GetString("guard.white_list"), ",")
  87. var preallocateSize int64
  88. if option.VolumePreallocate {
  89. preallocateSize = int64(option.VolumeSizeLimitMB) * (1 << 20)
  90. }
  91. grpcDialOption := security.LoadClientTLS(v, "grpc.master")
  92. ms := &MasterServer{
  93. option: option,
  94. preallocateSize: preallocateSize,
  95. volumeGrowthRequestChan: make(chan *topology.VolumeGrowRequest, 1<<6),
  96. clientChans: make(map[string]chan *master_pb.KeepConnectedResponse),
  97. grpcDialOption: grpcDialOption,
  98. MasterClient: wdclient.NewMasterClient(grpcDialOption, "", cluster.MasterType, option.Master, "", "", *pb.NewServiceDiscoveryFromMap(peers)),
  99. adminLocks: NewAdminLocks(),
  100. Cluster: cluster.NewCluster(),
  101. }
  102. ms.boundedLeaderChan = make(chan int, 16)
  103. ms.MasterClient.SetOnPeerUpdateFn(ms.OnPeerUpdate)
  104. seq := ms.createSequencer(option)
  105. if nil == seq {
  106. glog.Fatalf("create sequencer failed.")
  107. }
  108. ms.Topo = topology.NewTopology("topo", seq, uint64(ms.option.VolumeSizeLimitMB)*1024*1024, 5, replicationAsMin)
  109. ms.vg = topology.NewDefaultVolumeGrowth()
  110. glog.V(0).Infoln("Volume Size Limit is", ms.option.VolumeSizeLimitMB, "MB")
  111. ms.guard = security.NewGuard(append(ms.option.WhiteList, whiteList...), signingKey, expiresAfterSec, readSigningKey, readExpiresAfterSec)
  112. handleStaticResources2(r)
  113. r.HandleFunc("/", ms.proxyToLeader(ms.uiStatusHandler))
  114. r.HandleFunc("/ui/index.html", ms.uiStatusHandler)
  115. if !ms.option.DisableHttp {
  116. r.HandleFunc("/dir/assign", ms.proxyToLeader(ms.guard.WhiteList(ms.dirAssignHandler)))
  117. r.HandleFunc("/dir/lookup", ms.guard.WhiteList(ms.dirLookupHandler))
  118. r.HandleFunc("/dir/status", ms.proxyToLeader(ms.guard.WhiteList(ms.dirStatusHandler)))
  119. r.HandleFunc("/col/delete", ms.proxyToLeader(ms.guard.WhiteList(ms.collectionDeleteHandler)))
  120. r.HandleFunc("/vol/grow", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeGrowHandler)))
  121. r.HandleFunc("/vol/status", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeStatusHandler)))
  122. r.HandleFunc("/vol/vacuum", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeVacuumHandler)))
  123. r.HandleFunc("/submit", ms.guard.WhiteList(ms.submitFromMasterServerHandler))
  124. r.HandleFunc("/collection/info", ms.guard.WhiteList(ms.collectionInfoHandler))
  125. /*
  126. r.HandleFunc("/stats/health", ms.guard.WhiteList(statsHealthHandler))
  127. r.HandleFunc("/stats/counter", ms.guard.WhiteList(statsCounterHandler))
  128. r.HandleFunc("/stats/memory", ms.guard.WhiteList(statsMemoryHandler))
  129. */
  130. r.HandleFunc("/{fileId}", ms.redirectHandler)
  131. }
  132. ms.Topo.StartRefreshWritableVolumes(
  133. ms.grpcDialOption,
  134. ms.option.GarbageThreshold,
  135. ms.option.MaxParallelVacuumPerServer,
  136. topology.VolumeGrowStrategy.Threshold,
  137. ms.preallocateSize,
  138. )
  139. ms.ProcessGrowRequest()
  140. if !option.IsFollower {
  141. ms.startAdminScripts()
  142. }
  143. return ms
  144. }
  145. func (ms *MasterServer) SetRaftServer(raftServer *RaftServer) {
  146. var raftServerName string
  147. ms.Topo.RaftServerAccessLock.Lock()
  148. if raftServer.raftServer != nil {
  149. ms.Topo.RaftServer = raftServer.raftServer
  150. ms.Topo.RaftServer.AddEventListener(raft.LeaderChangeEventType, func(e raft.Event) {
  151. glog.V(0).Infof("leader change event: %+v => %+v", e.PrevValue(), e.Value())
  152. stats.MasterLeaderChangeCounter.WithLabelValues(fmt.Sprintf("%+v", e.Value())).Inc()
  153. if ms.Topo.RaftServer.Leader() != "" {
  154. glog.V(0).Infof("[%s] %s becomes leader.", ms.Topo.RaftServer.Name(), ms.Topo.RaftServer.Leader())
  155. ms.Topo.LastLeaderChangeTime = time.Now()
  156. }
  157. })
  158. raftServerName = fmt.Sprintf("[%s]", ms.Topo.RaftServer.Name())
  159. } else if raftServer.RaftHashicorp != nil {
  160. ms.Topo.HashicorpRaft = raftServer.RaftHashicorp
  161. raftServerName = ms.Topo.HashicorpRaft.String()
  162. ms.Topo.LastLeaderChangeTime = time.Now()
  163. }
  164. ms.Topo.RaftServerAccessLock.Unlock()
  165. if ms.Topo.IsLeader() {
  166. glog.V(0).Infof("%s I am the leader!", raftServerName)
  167. } else {
  168. var raftServerLeader string
  169. ms.Topo.RaftServerAccessLock.RLock()
  170. if ms.Topo.RaftServer != nil {
  171. raftServerLeader = ms.Topo.RaftServer.Leader()
  172. } else if ms.Topo.HashicorpRaft != nil {
  173. raftServerName = ms.Topo.HashicorpRaft.String()
  174. raftServerLeaderAddr, _ := ms.Topo.HashicorpRaft.LeaderWithID()
  175. raftServerLeader = string(raftServerLeaderAddr)
  176. }
  177. ms.Topo.RaftServerAccessLock.RUnlock()
  178. glog.V(0).Infof("%s %s - is the leader.", raftServerName, raftServerLeader)
  179. }
  180. }
  181. func (ms *MasterServer) proxyToLeader(f http.HandlerFunc) http.HandlerFunc {
  182. return func(w http.ResponseWriter, r *http.Request) {
  183. if ms.Topo.IsLeader() {
  184. f(w, r)
  185. return
  186. }
  187. // get the current raft leader
  188. leaderAddr, _ := ms.Topo.MaybeLeader()
  189. raftServerLeader := leaderAddr.ToHttpAddress()
  190. if raftServerLeader == "" {
  191. f(w, r)
  192. return
  193. }
  194. ms.boundedLeaderChan <- 1
  195. defer func() { <-ms.boundedLeaderChan }()
  196. targetUrl, err := url.Parse("http://" + raftServerLeader)
  197. if err != nil {
  198. writeJsonError(w, r, http.StatusInternalServerError,
  199. fmt.Errorf("Leader URL http://%s Parse Error: %v", raftServerLeader, err))
  200. return
  201. }
  202. // proxy to leader
  203. glog.V(4).Infoln("proxying to leader", raftServerLeader)
  204. proxy := httputil.NewSingleHostReverseProxy(targetUrl)
  205. director := proxy.Director
  206. proxy.Director = func(req *http.Request) {
  207. actualHost, err := security.GetActualRemoteHost(req)
  208. if err == nil {
  209. req.Header.Set("HTTP_X_FORWARDED_FOR", actualHost)
  210. }
  211. director(req)
  212. }
  213. proxy.Transport = util_http.GetGlobalHttpClient().GetClientTransport()
  214. proxy.ServeHTTP(w, r)
  215. }
  216. }
  217. func (ms *MasterServer) startAdminScripts() {
  218. v := util.GetViper()
  219. adminScripts := v.GetString("master.maintenance.scripts")
  220. if adminScripts == "" {
  221. return
  222. }
  223. glog.V(0).Infof("adminScripts: %v", adminScripts)
  224. v.SetDefault("master.maintenance.sleep_minutes", 17)
  225. sleepMinutes := v.GetInt("master.maintenance.sleep_minutes")
  226. scriptLines := strings.Split(adminScripts, "\n")
  227. if !strings.Contains(adminScripts, "lock") {
  228. scriptLines = append(append([]string{}, "lock"), scriptLines...)
  229. scriptLines = append(scriptLines, "unlock")
  230. }
  231. masterAddress := string(ms.option.Master)
  232. var shellOptions shell.ShellOptions
  233. shellOptions.GrpcDialOption = security.LoadClientTLS(v, "grpc.master")
  234. shellOptions.Masters = &masterAddress
  235. shellOptions.Directory = "/"
  236. emptyFilerGroup := ""
  237. shellOptions.FilerGroup = &emptyFilerGroup
  238. commandEnv := shell.NewCommandEnv(&shellOptions)
  239. reg, _ := regexp.Compile(`'.*?'|".*?"|\S+`)
  240. go commandEnv.MasterClient.KeepConnectedToMaster(context.Background())
  241. go func() {
  242. for {
  243. time.Sleep(time.Duration(sleepMinutes) * time.Minute)
  244. if ms.Topo.IsLeader() && ms.MasterClient.GetMaster(context.Background()) != "" {
  245. shellOptions.FilerAddress = ms.GetOneFiler(cluster.FilerGroupName(*shellOptions.FilerGroup))
  246. if shellOptions.FilerAddress == "" {
  247. continue
  248. }
  249. for _, line := range scriptLines {
  250. for _, c := range strings.Split(line, ";") {
  251. processEachCmd(reg, c, commandEnv)
  252. }
  253. }
  254. }
  255. }
  256. }()
  257. }
  258. func processEachCmd(reg *regexp.Regexp, line string, commandEnv *shell.CommandEnv) {
  259. cmds := reg.FindAllString(line, -1)
  260. if len(cmds) == 0 {
  261. return
  262. }
  263. args := make([]string, len(cmds[1:]))
  264. for i := range args {
  265. args[i] = strings.Trim(string(cmds[1+i]), "\"'")
  266. }
  267. cmd := cmds[0]
  268. for _, c := range shell.Commands {
  269. if c.Name() == cmd {
  270. if c.HasTag(shell.ResourceHeavy) {
  271. glog.Warningf("%s is resource heavy and should not run on master", cmd)
  272. continue
  273. }
  274. glog.V(0).Infof("executing: %s %v", cmd, args)
  275. if err := c.Do(args, commandEnv, os.Stdout); err != nil {
  276. glog.V(0).Infof("error: %v", err)
  277. }
  278. }
  279. }
  280. }
  281. func (ms *MasterServer) createSequencer(option *MasterOption) sequence.Sequencer {
  282. var seq sequence.Sequencer
  283. v := util.GetViper()
  284. seqType := strings.ToLower(v.GetString(SequencerType))
  285. glog.V(1).Infof("[%s] : [%s]", SequencerType, seqType)
  286. switch strings.ToLower(seqType) {
  287. case "snowflake":
  288. var err error
  289. snowflakeId := v.GetInt(SequencerSnowflakeId)
  290. seq, err = sequence.NewSnowflakeSequencer(string(option.Master), snowflakeId)
  291. if err != nil {
  292. glog.Error(err)
  293. seq = nil
  294. }
  295. case "raft":
  296. fallthrough
  297. default:
  298. seq = sequence.NewMemorySequencer()
  299. }
  300. return seq
  301. }
  302. func (ms *MasterServer) OnPeerUpdate(update *master_pb.ClusterNodeUpdate, startFrom time.Time) {
  303. ms.Topo.RaftServerAccessLock.RLock()
  304. defer ms.Topo.RaftServerAccessLock.RUnlock()
  305. if update.NodeType != cluster.MasterType || ms.Topo.HashicorpRaft == nil {
  306. return
  307. }
  308. glog.V(4).Infof("OnPeerUpdate: %+v", update)
  309. peerAddress := pb.ServerAddress(update.Address)
  310. peerName := string(peerAddress)
  311. if ms.Topo.HashicorpRaft.State() != hashicorpRaft.Leader {
  312. return
  313. }
  314. if update.IsAdd {
  315. raftServerFound := false
  316. for _, server := range ms.Topo.HashicorpRaft.GetConfiguration().Configuration().Servers {
  317. if string(server.ID) == peerName {
  318. raftServerFound = true
  319. }
  320. }
  321. if !raftServerFound {
  322. glog.V(0).Infof("adding new raft server: %s", peerName)
  323. ms.Topo.HashicorpRaft.AddVoter(
  324. hashicorpRaft.ServerID(peerName),
  325. hashicorpRaft.ServerAddress(peerAddress.ToGrpcAddress()), 0, 0)
  326. }
  327. } else {
  328. pb.WithMasterClient(false, peerAddress, ms.grpcDialOption, true, func(client master_pb.SeaweedClient) error {
  329. ctx, cancel := context.WithTimeout(context.TODO(), 15*time.Second)
  330. defer cancel()
  331. if _, err := client.Ping(ctx, &master_pb.PingRequest{Target: string(peerAddress), TargetType: cluster.MasterType}); err != nil {
  332. glog.V(0).Infof("master %s didn't respond to pings. remove raft server", peerName)
  333. if err := ms.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
  334. _, err := client.RaftRemoveServer(context.Background(), &master_pb.RaftRemoveServerRequest{
  335. Id: peerName,
  336. Force: false,
  337. })
  338. return err
  339. }); err != nil {
  340. glog.Warningf("failed removing old raft server: %v", err)
  341. return err
  342. }
  343. } else {
  344. glog.V(0).Infof("master %s successfully responded to ping", peerName)
  345. }
  346. return nil
  347. })
  348. }
  349. }
  350. func (ms *MasterServer) Shutdown() {
  351. if ms.Topo == nil || ms.Topo.HashicorpRaft == nil {
  352. return
  353. }
  354. if ms.Topo.HashicorpRaft.State() == hashicorpRaft.Leader {
  355. ms.Topo.HashicorpRaft.LeadershipTransfer()
  356. }
  357. ms.Topo.HashicorpRaft.Shutdown()
  358. }
  359. func (ms *MasterServer) Reload() {
  360. glog.V(0).Infoln("Reload master server...")
  361. util.LoadConfiguration("security", false)
  362. v := util.GetViper()
  363. ms.guard.UpdateWhiteList(append(ms.option.WhiteList,
  364. util.StringSplit(v.GetString("guard.white_list"), ",")...),
  365. )
  366. }