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.

274 lines
8.6 KiB

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