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.

325 lines
10 KiB

6 years ago
3 years ago
6 years ago
5 years ago
5 years ago
6 years ago
6 years ago
6 years ago
5 years ago
3 years ago
6 years ago
6 years ago
4 years ago
3 years ago
3 years ago
6 years ago
6 years ago
6 years ago
9 years ago
  1. package weed_server
  2. import (
  3. "fmt"
  4. "github.com/chrislusf/seaweedfs/weed/stats"
  5. "net/http"
  6. "net/http/httputil"
  7. "net/url"
  8. "os"
  9. "regexp"
  10. "strings"
  11. "sync"
  12. "time"
  13. "github.com/chrislusf/seaweedfs/weed/cluster"
  14. "github.com/chrislusf/seaweedfs/weed/pb"
  15. "github.com/chrislusf/raft"
  16. "github.com/gorilla/mux"
  17. "google.golang.org/grpc"
  18. "github.com/chrislusf/seaweedfs/weed/glog"
  19. "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
  20. "github.com/chrislusf/seaweedfs/weed/security"
  21. "github.com/chrislusf/seaweedfs/weed/sequence"
  22. "github.com/chrislusf/seaweedfs/weed/shell"
  23. "github.com/chrislusf/seaweedfs/weed/topology"
  24. "github.com/chrislusf/seaweedfs/weed/util"
  25. "github.com/chrislusf/seaweedfs/weed/wdclient"
  26. )
  27. const (
  28. SequencerType = "master.sequencer.type"
  29. SequencerSnowflakeId = "master.sequencer.sequencer_snowflake_id"
  30. )
  31. type MasterOption struct {
  32. Master pb.ServerAddress
  33. MetaFolder string
  34. VolumeSizeLimitMB uint32
  35. VolumePreallocate bool
  36. // PulseSeconds int
  37. DefaultReplicaPlacement string
  38. GarbageThreshold float64
  39. WhiteList []string
  40. DisableHttp bool
  41. MetricsAddress string
  42. MetricsIntervalSec int
  43. IsFollower bool
  44. }
  45. type MasterServer struct {
  46. master_pb.UnimplementedSeaweedServer
  47. option *MasterOption
  48. guard *security.Guard
  49. preallocateSize int64
  50. Topo *topology.Topology
  51. vg *topology.VolumeGrowth
  52. vgCh chan *topology.VolumeGrowRequest
  53. boundedLeaderChan chan int
  54. // notifying clients
  55. clientChansLock sync.RWMutex
  56. clientChans map[string]chan *master_pb.KeepConnectedResponse
  57. grpcDialOption grpc.DialOption
  58. MasterClient *wdclient.MasterClient
  59. adminLocks *AdminLocks
  60. Cluster *cluster.Cluster
  61. }
  62. func NewMasterServer(r *mux.Router, option *MasterOption, peers map[string]pb.ServerAddress) *MasterServer {
  63. v := util.GetViper()
  64. signingKey := v.GetString("jwt.signing.key")
  65. v.SetDefault("jwt.signing.expires_after_seconds", 10)
  66. expiresAfterSec := v.GetInt("jwt.signing.expires_after_seconds")
  67. readSigningKey := v.GetString("jwt.signing.read.key")
  68. v.SetDefault("jwt.signing.read.expires_after_seconds", 60)
  69. readExpiresAfterSec := v.GetInt("jwt.signing.read.expires_after_seconds")
  70. v.SetDefault("master.replication.treat_replication_as_minimums", false)
  71. replicationAsMin := v.GetBool("master.replication.treat_replication_as_minimums")
  72. v.SetDefault("master.volume_growth.copy_1", 7)
  73. v.SetDefault("master.volume_growth.copy_2", 6)
  74. v.SetDefault("master.volume_growth.copy_3", 3)
  75. v.SetDefault("master.volume_growth.copy_other", 1)
  76. v.SetDefault("master.volume_growth.threshold", 0.9)
  77. var preallocateSize int64
  78. if option.VolumePreallocate {
  79. preallocateSize = int64(option.VolumeSizeLimitMB) * (1 << 20)
  80. }
  81. grpcDialOption := security.LoadClientTLS(v, "grpc.master")
  82. ms := &MasterServer{
  83. option: option,
  84. preallocateSize: preallocateSize,
  85. vgCh: make(chan *topology.VolumeGrowRequest, 1<<6),
  86. clientChans: make(map[string]chan *master_pb.KeepConnectedResponse),
  87. grpcDialOption: grpcDialOption,
  88. MasterClient: wdclient.NewMasterClient(grpcDialOption, cluster.MasterType, option.Master, "", peers),
  89. adminLocks: NewAdminLocks(),
  90. Cluster: cluster.NewCluster(),
  91. }
  92. ms.boundedLeaderChan = make(chan int, 16)
  93. seq := ms.createSequencer(option)
  94. if nil == seq {
  95. glog.Fatalf("create sequencer failed.")
  96. }
  97. ms.Topo = topology.NewTopology("topo", seq, uint64(ms.option.VolumeSizeLimitMB)*1024*1024, 5, replicationAsMin)
  98. ms.vg = topology.NewDefaultVolumeGrowth()
  99. glog.V(0).Infoln("Volume Size Limit is", ms.option.VolumeSizeLimitMB, "MB")
  100. ms.guard = security.NewGuard(ms.option.WhiteList, signingKey, expiresAfterSec, readSigningKey, readExpiresAfterSec)
  101. handleStaticResources2(r)
  102. r.HandleFunc("/", ms.proxyToLeader(ms.uiStatusHandler))
  103. r.HandleFunc("/ui/index.html", ms.uiStatusHandler)
  104. if !ms.option.DisableHttp {
  105. r.HandleFunc("/dir/assign", ms.proxyToLeader(ms.guard.WhiteList(ms.dirAssignHandler)))
  106. r.HandleFunc("/dir/lookup", ms.guard.WhiteList(ms.dirLookupHandler))
  107. r.HandleFunc("/dir/status", ms.proxyToLeader(ms.guard.WhiteList(ms.dirStatusHandler)))
  108. r.HandleFunc("/col/delete", ms.proxyToLeader(ms.guard.WhiteList(ms.collectionDeleteHandler)))
  109. r.HandleFunc("/vol/grow", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeGrowHandler)))
  110. r.HandleFunc("/vol/status", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeStatusHandler)))
  111. r.HandleFunc("/vol/vacuum", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeVacuumHandler)))
  112. r.HandleFunc("/submit", ms.guard.WhiteList(ms.submitFromMasterServerHandler))
  113. /*
  114. r.HandleFunc("/stats/health", ms.guard.WhiteList(statsHealthHandler))
  115. r.HandleFunc("/stats/counter", ms.guard.WhiteList(statsCounterHandler))
  116. r.HandleFunc("/stats/memory", ms.guard.WhiteList(statsMemoryHandler))
  117. */
  118. r.HandleFunc("/{fileId}", ms.redirectHandler)
  119. }
  120. ms.Topo.StartRefreshWritableVolumes(
  121. ms.grpcDialOption,
  122. ms.option.GarbageThreshold,
  123. v.GetFloat64("master.volume_growth.threshold"),
  124. ms.preallocateSize,
  125. )
  126. ms.ProcessGrowRequest()
  127. if !option.IsFollower {
  128. ms.startAdminScripts()
  129. }
  130. return ms
  131. }
  132. func (ms *MasterServer) SetRaftServer(raftServer *RaftServer) {
  133. var raftServerName string
  134. if raftServer.raftServer != nil {
  135. ms.Topo.RaftServer = raftServer.raftServer
  136. ms.Topo.RaftServer.AddEventListener(raft.LeaderChangeEventType, func(e raft.Event) {
  137. glog.V(0).Infof("leader change event: %+v => %+v", e.PrevValue(), e.Value())
  138. stats.MasterLeaderChangeCounter.WithLabelValues(fmt.Sprintf("%+v", e.Value())).Inc()
  139. if ms.Topo.RaftServer.Leader() != "" {
  140. glog.V(0).Infoln("[", ms.Topo.RaftServer.Name(), "]", ms.Topo.RaftServer.Leader(), "becomes leader.")
  141. }
  142. })
  143. raftServerName = ms.Topo.RaftServer.Name()
  144. } else if raftServer.RaftHashicorp != nil {
  145. ms.Topo.HashicorpRaft = raftServer.RaftHashicorp
  146. leaderCh := raftServer.RaftHashicorp.LeaderCh()
  147. prevLeader := ms.Topo.HashicorpRaft.Leader()
  148. go func() {
  149. for {
  150. select {
  151. case isLeader := <-leaderCh:
  152. leader := ms.Topo.HashicorpRaft.Leader()
  153. glog.V(0).Infof("is leader %+v change event: %+v => %+v", isLeader, prevLeader, leader)
  154. stats.MasterLeaderChangeCounter.WithLabelValues(fmt.Sprintf("%+v", leader)).Inc()
  155. prevLeader = leader
  156. }
  157. }
  158. }()
  159. raftServerName = ms.Topo.HashicorpRaft.String()
  160. }
  161. if ms.Topo.IsLeader() {
  162. glog.V(0).Infoln("[", raftServerName, "]", "I am the leader!")
  163. } else {
  164. if ms.Topo.RaftServer != nil && ms.Topo.RaftServer.Leader() != "" {
  165. glog.V(0).Infoln("[", ms.Topo.RaftServer.Name(), "]", ms.Topo.RaftServer.Leader(), "is the leader.")
  166. } else if ms.Topo.HashicorpRaft != nil && ms.Topo.HashicorpRaft.Leader() != "" {
  167. glog.V(0).Infoln("[", ms.Topo.HashicorpRaft.String(), "]", ms.Topo.HashicorpRaft.Leader(), "is the leader.")
  168. }
  169. }
  170. }
  171. func (ms *MasterServer) proxyToLeader(f http.HandlerFunc) http.HandlerFunc {
  172. return func(w http.ResponseWriter, r *http.Request) {
  173. if ms.Topo.IsLeader() {
  174. f(w, r)
  175. } else if ms.Topo.RaftServer != nil && ms.Topo.RaftServer.Leader() != "" {
  176. ms.boundedLeaderChan <- 1
  177. defer func() { <-ms.boundedLeaderChan }()
  178. targetUrl, err := url.Parse("http://" + ms.Topo.RaftServer.Leader())
  179. if err != nil {
  180. writeJsonError(w, r, http.StatusInternalServerError,
  181. fmt.Errorf("Leader URL http://%s Parse Error: %v", ms.Topo.RaftServer.Leader(), err))
  182. return
  183. }
  184. glog.V(4).Infoln("proxying to leader", ms.Topo.RaftServer.Leader())
  185. proxy := httputil.NewSingleHostReverseProxy(targetUrl)
  186. director := proxy.Director
  187. proxy.Director = func(req *http.Request) {
  188. actualHost, err := security.GetActualRemoteHost(req)
  189. if err == nil {
  190. req.Header.Set("HTTP_X_FORWARDED_FOR", actualHost)
  191. }
  192. director(req)
  193. }
  194. proxy.Transport = util.Transport
  195. proxy.ServeHTTP(w, r)
  196. } else {
  197. // handle requests locally
  198. f(w, r)
  199. }
  200. }
  201. }
  202. func (ms *MasterServer) startAdminScripts() {
  203. v := util.GetViper()
  204. adminScripts := v.GetString("master.maintenance.scripts")
  205. if adminScripts == "" {
  206. return
  207. }
  208. glog.V(0).Infof("adminScripts: %v", adminScripts)
  209. v.SetDefault("master.maintenance.sleep_minutes", 17)
  210. sleepMinutes := v.GetInt("master.maintenance.sleep_minutes")
  211. scriptLines := strings.Split(adminScripts, "\n")
  212. if !strings.Contains(adminScripts, "lock") {
  213. scriptLines = append(append([]string{}, "lock"), scriptLines...)
  214. scriptLines = append(scriptLines, "unlock")
  215. }
  216. masterAddress := string(ms.option.Master)
  217. var shellOptions shell.ShellOptions
  218. shellOptions.GrpcDialOption = security.LoadClientTLS(v, "grpc.master")
  219. shellOptions.Masters = &masterAddress
  220. shellOptions.Directory = "/"
  221. commandEnv := shell.NewCommandEnv(&shellOptions)
  222. reg, _ := regexp.Compile(`'.*?'|".*?"|\S+`)
  223. go commandEnv.MasterClient.KeepConnectedToMaster()
  224. go func() {
  225. commandEnv.MasterClient.WaitUntilConnected()
  226. for {
  227. time.Sleep(time.Duration(sleepMinutes) * time.Minute)
  228. if ms.Topo.IsLeader() {
  229. shellOptions.FilerAddress = ms.GetOneFiler()
  230. if shellOptions.FilerAddress == "" {
  231. continue
  232. }
  233. for _, line := range scriptLines {
  234. for _, c := range strings.Split(line, ";") {
  235. processEachCmd(reg, c, commandEnv)
  236. }
  237. }
  238. }
  239. }
  240. }()
  241. }
  242. func processEachCmd(reg *regexp.Regexp, line string, commandEnv *shell.CommandEnv) {
  243. cmds := reg.FindAllString(line, -1)
  244. if len(cmds) == 0 {
  245. return
  246. }
  247. args := make([]string, len(cmds[1:]))
  248. for i := range args {
  249. args[i] = strings.Trim(string(cmds[1+i]), "\"'")
  250. }
  251. cmd := strings.ToLower(cmds[0])
  252. for _, c := range shell.Commands {
  253. if c.Name() == cmd {
  254. glog.V(0).Infof("executing: %s %v", cmd, args)
  255. if err := c.Do(args, commandEnv, os.Stdout); err != nil {
  256. glog.V(0).Infof("error: %v", err)
  257. }
  258. }
  259. }
  260. }
  261. func (ms *MasterServer) createSequencer(option *MasterOption) sequence.Sequencer {
  262. var seq sequence.Sequencer
  263. v := util.GetViper()
  264. seqType := strings.ToLower(v.GetString(SequencerType))
  265. glog.V(1).Infof("[%s] : [%s]", SequencerType, seqType)
  266. switch strings.ToLower(seqType) {
  267. case "snowflake":
  268. var err error
  269. snowflakeId := v.GetInt(SequencerSnowflakeId)
  270. seq, err = sequence.NewSnowflakeSequencer(string(option.Master), snowflakeId)
  271. if err != nil {
  272. glog.Error(err)
  273. seq = nil
  274. }
  275. default:
  276. seq = sequence.NewMemorySequencer()
  277. }
  278. return seq
  279. }