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.

307 lines
9.5 KiB

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