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.

264 lines
8.3 KiB

7 years ago
5 years ago
6 years ago
6 years ago
6 years ago
6 years ago
6 years ago
6 years ago
6 years ago
6 years ago
9 years ago
6 years ago
5 years ago
5 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. "strconv"
  11. "strings"
  12. "sync"
  13. "time"
  14. "github.com/chrislusf/raft"
  15. "github.com/chrislusf/seaweedfs/weed/glog"
  16. "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
  17. "github.com/chrislusf/seaweedfs/weed/security"
  18. "github.com/chrislusf/seaweedfs/weed/sequence"
  19. "github.com/chrislusf/seaweedfs/weed/shell"
  20. "github.com/chrislusf/seaweedfs/weed/topology"
  21. "github.com/chrislusf/seaweedfs/weed/util"
  22. "github.com/chrislusf/seaweedfs/weed/wdclient"
  23. "github.com/gorilla/mux"
  24. "github.com/spf13/viper"
  25. "google.golang.org/grpc"
  26. )
  27. const (
  28. MasterPrefix = "master.maintenance"
  29. SequencerType = MasterPrefix + ".sequencer_type"
  30. SequencerEtcdUrls = MasterPrefix + ".sequencer_etcd_urls"
  31. )
  32. type MasterOption struct {
  33. Port int
  34. MetaFolder string
  35. VolumeSizeLimitMB uint
  36. VolumePreallocate bool
  37. PulseSeconds int
  38. DefaultReplicaPlacement string
  39. GarbageThreshold float64
  40. WhiteList []string
  41. DisableHttp bool
  42. MetricsAddress string
  43. MetricsIntervalSec int
  44. }
  45. type MasterServer struct {
  46. option *MasterOption
  47. guard *security.Guard
  48. preallocateSize int64
  49. Topo *topology.Topology
  50. vg *topology.VolumeGrowth
  51. vgLock sync.Mutex
  52. bounedLeaderChan chan int
  53. // notifying clients
  54. clientChansLock sync.RWMutex
  55. clientChans map[string]chan *master_pb.VolumeLocation
  56. grpcDialOption grpc.DialOption
  57. MasterClient *wdclient.MasterClient
  58. }
  59. func NewMasterServer(r *mux.Router, option *MasterOption, peers []string) *MasterServer {
  60. v := viper.GetViper()
  61. signingKey := v.GetString("jwt.signing.key")
  62. v.SetDefault("jwt.signing.expires_after_seconds", 10)
  63. expiresAfterSec := v.GetInt("jwt.signing.expires_after_seconds")
  64. readSigningKey := v.GetString("jwt.signing.read.key")
  65. v.SetDefault("jwt.signing.read.expires_after_seconds", 60)
  66. readExpiresAfterSec := v.GetInt("jwt.signing.read.expires_after_seconds")
  67. var preallocateSize int64
  68. if option.VolumePreallocate {
  69. preallocateSize = int64(option.VolumeSizeLimitMB) * (1 << 20)
  70. }
  71. grpcDialOption := security.LoadClientTLS(v.Sub("grpc"), "master")
  72. ms := &MasterServer{
  73. option: option,
  74. preallocateSize: preallocateSize,
  75. clientChans: make(map[string]chan *master_pb.VolumeLocation),
  76. grpcDialOption: grpcDialOption,
  77. MasterClient: wdclient.NewMasterClient(context.Background(), grpcDialOption, "master", peers),
  78. }
  79. ms.bounedLeaderChan = make(chan int, 16)
  80. seq := ms.createSequencer(option)
  81. if nil == seq {
  82. glog.Fatalf("create sequencer failed.")
  83. }
  84. ms.Topo = topology.NewTopology("topo", seq, uint64(ms.option.VolumeSizeLimitMB)*1024*1024, ms.option.PulseSeconds)
  85. ms.vg = topology.NewDefaultVolumeGrowth()
  86. glog.V(0).Infoln("Volume Size Limit is", ms.option.VolumeSizeLimitMB, "MB")
  87. ms.guard = security.NewGuard(ms.option.WhiteList, signingKey, expiresAfterSec, readSigningKey, readExpiresAfterSec)
  88. if !ms.option.DisableHttp {
  89. handleStaticResources2(r)
  90. r.HandleFunc("/", ms.proxyToLeader(ms.uiStatusHandler))
  91. r.HandleFunc("/ui/index.html", ms.uiStatusHandler)
  92. r.HandleFunc("/dir/assign", ms.proxyToLeader(ms.guard.WhiteList(ms.dirAssignHandler)))
  93. r.HandleFunc("/dir/lookup", ms.guard.WhiteList(ms.dirLookupHandler))
  94. r.HandleFunc("/dir/status", ms.proxyToLeader(ms.guard.WhiteList(ms.dirStatusHandler)))
  95. r.HandleFunc("/col/delete", ms.proxyToLeader(ms.guard.WhiteList(ms.collectionDeleteHandler)))
  96. r.HandleFunc("/vol/grow", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeGrowHandler)))
  97. r.HandleFunc("/vol/status", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeStatusHandler)))
  98. r.HandleFunc("/vol/vacuum", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeVacuumHandler)))
  99. r.HandleFunc("/submit", ms.guard.WhiteList(ms.submitFromMasterServerHandler))
  100. r.HandleFunc("/stats/health", ms.guard.WhiteList(statsHealthHandler))
  101. r.HandleFunc("/stats/counter", ms.guard.WhiteList(statsCounterHandler))
  102. r.HandleFunc("/stats/memory", ms.guard.WhiteList(statsMemoryHandler))
  103. r.HandleFunc("/{fileId}", ms.redirectHandler)
  104. }
  105. ms.Topo.StartRefreshWritableVolumes(ms.grpcDialOption, ms.option.GarbageThreshold, ms.preallocateSize)
  106. ms.startAdminScripts()
  107. return ms
  108. }
  109. func (ms *MasterServer) SetRaftServer(raftServer *RaftServer) {
  110. ms.Topo.RaftServer = raftServer.raftServer
  111. ms.Topo.RaftServer.AddEventListener(raft.LeaderChangeEventType, func(e raft.Event) {
  112. glog.V(0).Infof("event: %+v", e)
  113. if ms.Topo.RaftServer.Leader() != "" {
  114. glog.V(0).Infoln("[", ms.Topo.RaftServer.Name(), "]", ms.Topo.RaftServer.Leader(), "becomes leader.")
  115. }
  116. })
  117. ms.Topo.RaftServer.AddEventListener(raft.StateChangeEventType, func(e raft.Event) {
  118. glog.V(0).Infof("state change: %+v", e)
  119. })
  120. if ms.Topo.IsLeader() {
  121. glog.V(0).Infoln("[", ms.Topo.RaftServer.Name(), "]", "I am the leader!")
  122. } else {
  123. if ms.Topo.RaftServer.Leader() != "" {
  124. glog.V(0).Infoln("[", ms.Topo.RaftServer.Name(), "]", ms.Topo.RaftServer.Leader(), "is the leader.")
  125. }
  126. }
  127. }
  128. func (ms *MasterServer) proxyToLeader(f func(w http.ResponseWriter, r *http.Request)) func(w http.ResponseWriter, r *http.Request) {
  129. return func(w http.ResponseWriter, r *http.Request) {
  130. if ms.Topo.IsLeader() {
  131. f(w, r)
  132. } else if ms.Topo.RaftServer != nil && ms.Topo.RaftServer.Leader() != "" {
  133. ms.bounedLeaderChan <- 1
  134. defer func() { <-ms.bounedLeaderChan }()
  135. targetUrl, err := url.Parse("http://" + ms.Topo.RaftServer.Leader())
  136. if err != nil {
  137. writeJsonError(w, r, http.StatusInternalServerError,
  138. fmt.Errorf("Leader URL http://%s Parse Error: %v", ms.Topo.RaftServer.Leader(), err))
  139. return
  140. }
  141. glog.V(4).Infoln("proxying to leader", ms.Topo.RaftServer.Leader())
  142. proxy := httputil.NewSingleHostReverseProxy(targetUrl)
  143. director := proxy.Director
  144. proxy.Director = func(req *http.Request) {
  145. actualHost, err := security.GetActualRemoteHost(req)
  146. if err == nil {
  147. req.Header.Set("HTTP_X_FORWARDED_FOR", actualHost)
  148. }
  149. director(req)
  150. }
  151. proxy.Transport = util.Transport
  152. proxy.ServeHTTP(w, r)
  153. } else {
  154. // drop it to the floor
  155. // writeJsonError(w, r, errors.New(ms.Topo.RaftServer.Name()+" does not know Leader yet:"+ms.Topo.RaftServer.Leader()))
  156. }
  157. }
  158. }
  159. func (ms *MasterServer) startAdminScripts() {
  160. v := viper.GetViper()
  161. adminScripts := v.GetString("master.maintenance.scripts")
  162. v.SetDefault("master.maintenance.sleep_minutes", 17)
  163. sleepMinutes := v.GetInt("master.maintenance.sleep_minutes")
  164. glog.V(0).Infof("adminScripts:\n%v", adminScripts)
  165. if adminScripts == "" {
  166. return
  167. }
  168. scriptLines := strings.Split(adminScripts, "\n")
  169. masterAddress := "localhost:" + strconv.Itoa(ms.option.Port)
  170. var shellOptions shell.ShellOptions
  171. shellOptions.GrpcDialOption = security.LoadClientTLS(viper.Sub("grpc"), "master")
  172. shellOptions.Masters = &masterAddress
  173. shellOptions.FilerHost = "localhost"
  174. shellOptions.FilerPort = 8888
  175. shellOptions.Directory = "/"
  176. commandEnv := shell.NewCommandEnv(shellOptions)
  177. reg, _ := regexp.Compile(`'.*?'|".*?"|\S+`)
  178. go commandEnv.MasterClient.KeepConnectedToMaster()
  179. go func() {
  180. commandEnv.MasterClient.WaitUntilConnected()
  181. c := time.Tick(time.Duration(sleepMinutes) * time.Minute)
  182. for _ = range c {
  183. if ms.Topo.IsLeader() {
  184. for _, line := range scriptLines {
  185. cmds := reg.FindAllString(line, -1)
  186. if len(cmds) == 0 {
  187. continue
  188. }
  189. args := make([]string, len(cmds[1:]))
  190. for i := range args {
  191. args[i] = strings.Trim(string(cmds[1+i]), "\"'")
  192. }
  193. cmd := strings.ToLower(cmds[0])
  194. for _, c := range shell.Commands {
  195. if c.Name() == cmd {
  196. glog.V(0).Infof("executing: %s %v", cmd, args)
  197. if err := c.Do(args, commandEnv, os.Stdout); err != nil {
  198. glog.V(0).Infof("error: %v", err)
  199. }
  200. }
  201. }
  202. }
  203. }
  204. }
  205. }()
  206. }
  207. func (ms *MasterServer) createSequencer(option *MasterOption) sequence.Sequencer {
  208. var seq sequence.Sequencer
  209. seqType := strings.ToLower(util.Config().GetString(SequencerType))
  210. glog.V(0).Infof("[%s] : [%s]", SequencerType, seqType)
  211. switch strings.ToLower(seqType) {
  212. case "memory":
  213. seq = sequence.NewMemorySequencer()
  214. case "etcd":
  215. var err error
  216. urls := util.Config().GetString(SequencerEtcdUrls)
  217. glog.V(0).Infof("[%s] : [%s]", SequencerEtcdUrls, urls)
  218. seq, err = sequence.NewEtcdSequencer(urls, option.MetaFolder)
  219. if err != nil {
  220. glog.Error(err)
  221. seq = nil
  222. }
  223. default:
  224. seq = sequence.NewMemorySequencer()
  225. }
  226. return seq
  227. }