fix conflict

This commit is contained in:
Xiang Li 2013-08-08 20:05:52 -07:00
commit fff8acaa22
18 changed files with 280 additions and 189 deletions

View File

@ -246,10 +246,10 @@ http://www.g-loaded.eu/2005/11/10/be-your-own-ca/
``` ```
```sh ```sh
./etcd -clientCert client.crt -clientKey client.key -i ./etcd -clientCert client.crt -clientKey client.key -f
``` ```
`-i` is to ignore the previously created default configuration file. `-f` forces new node configuration if existing configuration is found (WARNING: data loss!)
`-clientCert` and `-clientKey` are the key and cert for transport layer security between client and server `-clientCert` and `-clientKey` are the key and cert for transport layer security between client and server
```sh ```sh
@ -276,7 +276,7 @@ And also the response from the etcd server.
We also can do authentication using CA cert. The clients will also need to provide their cert to the server. The server will check whether the cert is signed by the CA and decide whether to serve the request. We also can do authentication using CA cert. The clients will also need to provide their cert to the server. The server will check whether the cert is signed by the CA and decide whether to serve the request.
```sh ```sh
./etcd -clientCert client.crt -clientKey client.key -clientCAFile clientCA.crt -i ./etcd -clientCert client.crt -clientKey client.key -clientCAFile clientCA.crt -f
``` ```
```-clientCAFile``` is the path to the CA cert. ```-clientCAFile``` is the path to the CA cert.

166
etcd.go
View File

@ -37,22 +37,10 @@ var machinesFile string
var cluster []string var cluster []string
var hostname string var argInfo Info
var clientPort int
var raftPort int
var webPort int
var serverCertFile string
var serverKeyFile string
var serverCAFile string
var clientCertFile string
var clientKeyFile string
var clientCAFile string
var dirPath string var dirPath string
var ignore bool var force bool
var maxSize int var maxSize int
@ -71,22 +59,22 @@ func init() {
flag.StringVar(&machines, "C", "", "the ip address and port of a existing machines in the cluster, sepearate by comma") flag.StringVar(&machines, "C", "", "the ip address and port of a existing machines in the cluster, sepearate by comma")
flag.StringVar(&machinesFile, "CF", "", "the file contains a list of existing machines in the cluster, seperate by comma") flag.StringVar(&machinesFile, "CF", "", "the file contains a list of existing machines in the cluster, seperate by comma")
flag.StringVar(&hostname, "h", "0.0.0.0", "the hostname of the local machine") flag.StringVar(&argInfo.Hostname, "h", "0.0.0.0", "the hostname of the local machine")
flag.IntVar(&clientPort, "c", 4001, "the port to communicate with clients") flag.IntVar(&argInfo.ClientPort, "c", 4001, "the port to communicate with clients")
flag.IntVar(&raftPort, "s", 7001, "the port to communicate with servers") flag.IntVar(&argInfo.RaftPort, "s", 7001, "the port to communicate with servers")
flag.IntVar(&webPort, "w", -1, "the port of web interface (-1 means do not start web interface)") flag.IntVar(&argInfo.WebPort, "w", -1, "the port of web interface (-1 means do not start web interface)")
flag.StringVar(&serverCAFile, "serverCAFile", "", "the path of the CAFile") flag.StringVar(&argInfo.ServerCAFile, "serverCAFile", "", "the path of the CAFile")
flag.StringVar(&serverCertFile, "serverCert", "", "the cert file of the server") flag.StringVar(&argInfo.ServerCertFile, "serverCert", "", "the cert file of the server")
flag.StringVar(&serverKeyFile, "serverKey", "", "the key file of the server") flag.StringVar(&argInfo.ServerKeyFile, "serverKey", "", "the key file of the server")
flag.StringVar(&clientCAFile, "clientCAFile", "", "the path of the client CAFile") flag.StringVar(&argInfo.ClientCAFile, "clientCAFile", "", "the path of the client CAFile")
flag.StringVar(&clientCertFile, "clientCert", "", "the cert file of the client") flag.StringVar(&argInfo.ClientCertFile, "clientCert", "", "the cert file of the client")
flag.StringVar(&clientKeyFile, "clientKey", "", "the key file of the client") flag.StringVar(&argInfo.ClientKeyFile, "clientKey", "", "the key file of the client")
flag.StringVar(&dirPath, "d", "/tmp/", "the directory to store log and snapshot") flag.StringVar(&dirPath, "d", ".", "the directory to store log and snapshot")
flag.BoolVar(&ignore, "i", false, "ignore the old configuration, create a new node") flag.BoolVar(&force, "f", false, "force new node configuration if existing is found (WARNING: data loss!)")
flag.BoolVar(&snapshot, "snapshot", false, "open or close snapshot") flag.BoolVar(&snapshot, "snapshot", false, "open or close snapshot")
@ -226,14 +214,14 @@ func main() {
startRaft(st) startRaft(st)
if webPort != -1 { if argInfo.WebPort != -1 {
// start web // start web
etcdStore.SetMessager(&storeMsg) etcdStore.SetMessager(storeMsg)
go webHelper() go webHelper()
go web.Start(raftServer, webPort) go web.Start(raftServer, argInfo.WebPort)
} }
startClientTransport(info.ClientPort, clientSt) startClientTransport(*info, clientSt)
} }
@ -280,9 +268,9 @@ func startRaft(securityType int) {
for { for {
command := &JoinCommand{ command := &JoinCommand{
Name: raftServer.Name(), Name: raftServer.Name(),
Hostname: hostname, Hostname: argInfo.Hostname,
RaftPort: raftPort, RaftPort: argInfo.RaftPort,
ClientPort: clientPort, ClientPort: argInfo.ClientPort,
} }
_, err := raftServer.Do(command) _, err := raftServer.Do(command)
if err == nil { if err == nil {
@ -340,7 +328,7 @@ func startRaft(securityType int) {
} }
// start to response to raft requests // start to response to raft requests
go startRaftTransport(info.RaftPort, securityType) go startRaftTransport(*info, securityType)
} }
@ -367,7 +355,7 @@ func createTransporter(st int) transporter {
case HTTPSANDVERIFY: case HTTPSANDVERIFY:
t.scheme = "https://" t.scheme = "https://"
tlsCert, err := tls.LoadX509KeyPair(serverCertFile, serverKeyFile) tlsCert, err := tls.LoadX509KeyPair(argInfo.ServerCertFile, argInfo.ServerKeyFile)
if err != nil { if err != nil {
fatal(err) fatal(err)
@ -394,7 +382,7 @@ func dialTimeout(network, addr string) (net.Conn, error) {
} }
// Start to listen and response raft command // Start to listen and response raft command
func startRaftTransport(port int, st int) { func startRaftTransport(info Info, st int) {
// internal commands // internal commands
http.HandleFunc("/join", JoinHttpHandler) http.HandleFunc("/join", JoinHttpHandler)
@ -408,30 +396,30 @@ func startRaftTransport(port int, st int) {
switch st { switch st {
case HTTP: case HTTP:
fmt.Printf("raft server [%s] listen on http port %v\n", hostname, port) fmt.Printf("raft server [%s] listen on http port %v\n", info.Hostname, info.RaftPort)
fatal(http.ListenAndServe(fmt.Sprintf(":%d", port), nil)) fatal(http.ListenAndServe(fmt.Sprintf(":%d", info.RaftPort), nil))
case HTTPS: case HTTPS:
fmt.Printf("raft server [%s] listen on https port %v\n", hostname, port) fmt.Printf("raft server [%s] listen on https port %v\n", info.Hostname, info.RaftPort)
fatal(http.ListenAndServeTLS(fmt.Sprintf(":%d", port), serverCertFile, serverKeyFile, nil)) fatal(http.ListenAndServeTLS(fmt.Sprintf(":%d", info.RaftPort), info.ServerCertFile, argInfo.ServerKeyFile, nil))
case HTTPSANDVERIFY: case HTTPSANDVERIFY:
server := &http.Server{ server := &http.Server{
TLSConfig: &tls.Config{ TLSConfig: &tls.Config{
ClientAuth: tls.RequireAndVerifyClientCert, ClientAuth: tls.RequireAndVerifyClientCert,
ClientCAs: createCertPool(serverCAFile), ClientCAs: createCertPool(info.ServerCAFile),
}, },
Addr: fmt.Sprintf(":%d", port), Addr: fmt.Sprintf(":%d", info.RaftPort),
} }
fmt.Printf("raft server [%s] listen on https port %v\n", hostname, port) fmt.Printf("raft server [%s] listen on https port %v\n", info.Hostname, info.RaftPort)
fatal(server.ListenAndServeTLS(serverCertFile, serverKeyFile)) fatal(server.ListenAndServeTLS(info.ServerCertFile, argInfo.ServerKeyFile))
} }
} }
// Start to listen and response client command // Start to listen and response client command
func startClientTransport(port int, st int) { func startClientTransport(info Info, st int) {
// external commands // external commands
http.HandleFunc("/"+version+"/keys/", Multiplexer) http.HandleFunc("/"+version+"/keys/", Multiplexer)
http.HandleFunc("/"+version+"/watch/", WatchHttpHandler) http.HandleFunc("/"+version+"/watch/", WatchHttpHandler)
@ -444,24 +432,24 @@ func startClientTransport(port int, st int) {
switch st { switch st {
case HTTP: case HTTP:
fmt.Printf("etcd [%s] listen on http port %v\n", hostname, clientPort) fmt.Printf("etcd [%s] listen on http port %v\n", info.Hostname, info.ClientPort)
fatal(http.ListenAndServe(fmt.Sprintf(":%d", port), nil)) fatal(http.ListenAndServe(fmt.Sprintf(":%d", info.ClientPort), nil))
case HTTPS: case HTTPS:
fmt.Printf("etcd [%s] listen on https port %v\n", hostname, clientPort) fmt.Printf("etcd [%s] listen on https port %v\n", info.Hostname, info.ClientPort)
http.ListenAndServeTLS(fmt.Sprintf(":%d", port), clientCertFile, clientKeyFile, nil) http.ListenAndServeTLS(fmt.Sprintf(":%d", info.ClientPort), info.ClientCertFile, info.ClientKeyFile, nil)
case HTTPSANDVERIFY: case HTTPSANDVERIFY:
server := &http.Server{ server := &http.Server{
TLSConfig: &tls.Config{ TLSConfig: &tls.Config{
ClientAuth: tls.RequireAndVerifyClientCert, ClientAuth: tls.RequireAndVerifyClientCert,
ClientCAs: createCertPool(clientCAFile), ClientCAs: createCertPool(info.ClientCAFile),
}, },
Addr: fmt.Sprintf(":%d", port), Addr: fmt.Sprintf(":%d", info.ClientPort),
} }
fmt.Printf("etcd [%s] listen on https port %v\n", hostname, clientPort) fmt.Printf("etcd [%s] listen on https port %v\n", info.Hostname, info.ClientPort)
fatal(server.ListenAndServeTLS(clientCertFile, clientKeyFile)) fatal(server.ListenAndServeTLS(info.ClientCertFile, info.ClientKeyFile))
} }
} }
@ -511,6 +499,30 @@ func securityType(source int) int {
return -1 return -1
} }
func parseInfo(path string) *Info {
file, err := os.Open(path)
if err != nil {
return nil
}
info := &Info{}
defer file.Close()
content, err := ioutil.ReadAll(file)
if err != nil {
fatalf("Unable to read info: %v", err)
return nil
}
if err = json.Unmarshal(content, &info); err != nil {
fatalf("Unable to parse info: %v", err)
return nil
}
return info
}
// Get the server info from previous conf file // Get the server info from previous conf file
// or from the user // or from the user
func getInfo(path string) *Info { func getInfo(path string) *Info {
@ -519,8 +531,7 @@ func getInfo(path string) *Info {
infoPath := fmt.Sprintf("%s/info", path) infoPath := fmt.Sprintf("%s/info", path)
// Delete the old configuration if exist // Delete the old configuration if exist
if ignore { if force {
logPath := fmt.Sprintf("%s/log", path) logPath := fmt.Sprintf("%s/log", path)
confPath := fmt.Sprintf("%s/conf", path) confPath := fmt.Sprintf("%s/conf", path)
snapshotPath := fmt.Sprintf("%s/snapshot", path) snapshotPath := fmt.Sprintf("%s/snapshot", path)
@ -528,54 +539,33 @@ func getInfo(path string) *Info {
os.Remove(logPath) os.Remove(logPath)
os.Remove(confPath) os.Remove(confPath)
os.RemoveAll(snapshotPath) os.RemoveAll(snapshotPath)
} }
if file, err := os.Open(infoPath); err == nil { info := parseInfo(infoPath)
info := &Info{} if info != nil {
if content, err := ioutil.ReadAll(file); err != nil { fmt.Printf("Found node configuration in '%s'. Ignoring flags.\n", infoPath)
fatalf("Unable to read info: %v", err)
} else {
if err = json.Unmarshal(content, &info); err != nil {
fatalf("Unable to parse info: %v", err)
}
}
file.Close()
return info return info
} else { }
// Otherwise ask user for info and write it to file. // Otherwise ask user for info and write it to file.
argInfo.Hostname = strings.TrimSpace(argInfo.Hostname)
hostname = strings.TrimSpace(hostname) if argInfo.Hostname == "" {
if hostname == "" {
fatal("Please give the address of the local machine") fatal("Please give the address of the local machine")
} }
fmt.Println("address ", hostname) info = &argInfo
info := &Info{
Hostname: hostname,
RaftPort: raftPort,
ClientPort: clientPort,
WebPort: webPort,
ClientCAFile: clientCAFile,
ClientCertFile: clientCertFile,
ClientKeyFile: clientKeyFile,
ServerCAFile: serverCAFile,
ServerKeyFile: serverKeyFile,
ServerCertFile: serverCertFile,
}
// Write to file. // Write to file.
content, _ := json.Marshal(info) content, _ := json.MarshalIndent(info, "", " ")
content = []byte(string(content) + "\n") content = []byte(string(content) + "\n")
if err := ioutil.WriteFile(infoPath, content, 0644); err != nil { if err := ioutil.WriteFile(infoPath, content, 0644); err != nil {
fatalf("Unable to write info to file: %v", err) fatalf("Unable to write info to file: %v", err)
} }
fmt.Printf("Wrote node configuration to '%s'.\n", infoPath)
return info return info
}
} }
// Create client auth certpool // Create client auth certpool

View File

@ -34,7 +34,7 @@ func TestKillLeader(t *testing.T) {
var totalTime time.Duration var totalTime time.Duration
leader := "0.0.0.0:7001" leader := "127.0.0.1:7001"
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
port, _ := strconv.Atoi(strings.Split(leader, ":")[1]) port, _ := strconv.Atoi(strings.Split(leader, ":")[1])
@ -134,7 +134,7 @@ func BenchmarkEtcdDirectCall(b *testing.B) {
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
resp, _ := http.Get("http://0.0.0.0:4001/test/speed") resp, _ := http.Get("http://127.0.0.1:4001/test/speed")
resp.Body.Close() resp.Body.Close()
} }

View File

@ -14,7 +14,7 @@ import (
func TestSingleNode(t *testing.T) { func TestSingleNode(t *testing.T) {
procAttr := new(os.ProcAttr) procAttr := new(os.ProcAttr)
procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr}
args := []string{"etcd", "-i", "-d=/tmp/node1"} args := []string{"etcd", "-h=127.0.0.1", "-f", "-d=/tmp/node1"}
process, err := os.StartProcess("etcd", args, procAttr) process, err := os.StartProcess("etcd", args, procAttr)
if err != nil { if err != nil {
@ -56,9 +56,9 @@ func TestSingleNode(t *testing.T) {
func TestSingleNodeRecovery(t *testing.T) { func TestSingleNodeRecovery(t *testing.T) {
procAttr := new(os.ProcAttr) procAttr := new(os.ProcAttr)
procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr}
args := []string{"etcd", "-d=/tmp/node1"} args := []string{"etcd", "-h=127.0.0.1", "-d=/tmp/node1"}
process, err := os.StartProcess("etcd", append(args, "-i"), procAttr) process, err := os.StartProcess("etcd", append(args, "-f"), procAttr)
if err != nil { if err != nil {
t.Fatal("start process failed:" + err.Error()) t.Fatal("start process failed:" + err.Error())
return return

View File

@ -91,7 +91,7 @@ func SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) {
func ClientHttpHandler(w http.ResponseWriter, req *http.Request) { func ClientHttpHandler(w http.ResponseWriter, req *http.Request) {
debugf("[recv] Get %s/client/ ", raftTransporter.scheme+raftServer.Name()) debugf("[recv] Get %s/client/ ", raftTransporter.scheme+raftServer.Name())
w.WriteHeader(http.StatusOK) w.WriteHeader(http.StatusOK)
client := hostname + ":" + strconv.Itoa(clientPort) client := argInfo.Hostname + ":" + strconv.Itoa(argInfo.ClientPort)
w.Write([]byte(client)) w.Write([]byte(client))
} }

View File

@ -35,7 +35,7 @@ type Store struct {
// The string channel to send messages to the outside world // The string channel to send messages to the outside world
// Now we use it to send changes to the hub of the web service // Now we use it to send changes to the hub of the web service
messager *chan string messager chan<- string
// A map to keep the recent response to the clients // A map to keep the recent response to the clients
ResponseMap map[string]*Response ResponseMap map[string]*Response
@ -141,7 +141,7 @@ func CreateStore(max int) *Store {
} }
// Set the messager of the store // Set the messager of the store
func (s *Store) SetMessager(messager *chan string) { func (s *Store) SetMessager(messager chan<- string) {
s.messager = messager s.messager = messager
} }
@ -205,7 +205,7 @@ func (s *Store) internalSet(key string, value string, expireTime time.Time, inde
} else { } else {
// If we want the permanent node to have expire time // If we want the permanent node to have expire time
// We need to create create a go routine with a channel // We need to create a go routine with a channel
if isExpire { if isExpire {
node.update = make(chan time.Time) node.update = make(chan time.Time)
go s.monitorExpiration(key, node.update, expireTime) go s.monitorExpiration(key, node.update, expireTime)
@ -224,8 +224,7 @@ func (s *Store) internalSet(key string, value string, expireTime time.Time, inde
// Send to the messager // Send to the messager
if s.messager != nil && err == nil { if s.messager != nil && err == nil {
s.messager <- string(msg)
*s.messager <- string(msg)
} }
s.addToResponseMap(index, &resp) s.addToResponseMap(index, &resp)
@ -257,8 +256,7 @@ func (s *Store) internalSet(key string, value string, expireTime time.Time, inde
// Send to the messager // Send to the messager
if s.messager != nil && err == nil { if s.messager != nil && err == nil {
s.messager <- string(msg)
*s.messager <- string(msg)
} }
s.addToResponseMap(index, &resp) s.addToResponseMap(index, &resp)
@ -440,8 +438,7 @@ func (s *Store) internalDelete(key string, index uint64) ([]byte, error) {
// notify the messager // notify the messager
if s.messager != nil && err == nil { if s.messager != nil && err == nil {
s.messager <- string(msg)
*s.messager <- string(msg)
} }
s.addToResponseMap(index, &resp) s.addToResponseMap(index, &resp)
@ -486,7 +483,7 @@ func (s *Store) TestAndSet(key string, prevValue string, value string, expireTim
// The watchHub will send response to the channel when any key under the prefix // The watchHub will send response to the channel when any key under the prefix
// changes [since the sinceIndex if given] // changes [since the sinceIndex if given]
func (s *Store) AddWatcher(prefix string, watcher *Watcher, sinceIndex uint64) error { func (s *Store) AddWatcher(prefix string, watcher *Watcher, sinceIndex uint64) error {
return s.watcher.addWatcher(prefix, watcher, sinceIndex, s.ResponseStartIndex, s.Index, &s.ResponseMap) return s.watcher.addWatcher(prefix, watcher, sinceIndex, s.ResponseStartIndex, s.Index, s.ResponseMap)
} }
// This function should be created as a go routine to delete the key-value pair // This function should be created as a go routine to delete the key-value pair
@ -526,8 +523,7 @@ func (s *Store) monitorExpiration(key string, update chan time.Time, expireTime
// notify the messager // notify the messager
if s.messager != nil && err == nil { if s.messager != nil && err == nil {
s.messager <- string(msg)
*s.messager <- string(msg)
} }
return return

View File

@ -36,14 +36,14 @@ func NewWatcher() *Watcher {
// Add a watcher to the watcherHub // Add a watcher to the watcherHub
func (w *WatcherHub) addWatcher(prefix string, watcher *Watcher, sinceIndex uint64, func (w *WatcherHub) addWatcher(prefix string, watcher *Watcher, sinceIndex uint64,
responseStartIndex uint64, currentIndex uint64, resMap *map[string]*Response) error { responseStartIndex uint64, currentIndex uint64, resMap map[string]*Response) error {
prefix = path.Clean("/" + prefix) prefix = path.Clean("/" + prefix)
if sinceIndex != 0 && sinceIndex >= responseStartIndex { if sinceIndex != 0 && sinceIndex >= responseStartIndex {
for i := sinceIndex; i <= currentIndex; i++ { for i := sinceIndex; i <= currentIndex; i++ {
if checkResponse(prefix, i, resMap) { if checkResponse(prefix, i, resMap) {
watcher.C <- (*resMap)[strconv.FormatUint(i, 10)] watcher.C <- resMap[strconv.FormatUint(i, 10)]
return nil return nil
} }
} }
@ -52,22 +52,18 @@ func (w *WatcherHub) addWatcher(prefix string, watcher *Watcher, sinceIndex uint
_, ok := w.watchers[prefix] _, ok := w.watchers[prefix]
if !ok { if !ok {
w.watchers[prefix] = make([]*Watcher, 0) w.watchers[prefix] = make([]*Watcher, 0)
w.watchers[prefix] = append(w.watchers[prefix], watcher)
} else {
w.watchers[prefix] = append(w.watchers[prefix], watcher)
} }
w.watchers[prefix] = append(w.watchers[prefix], watcher)
return nil return nil
} }
// Check if the response has what we are watching // Check if the response has what we are watching
func checkResponse(prefix string, index uint64, resMap *map[string]*Response) bool { func checkResponse(prefix string, index uint64, resMap map[string]*Response) bool {
resp, ok := (*resMap)[strconv.FormatUint(index, 10)] resp, ok := resMap[strconv.FormatUint(index, 10)]
if !ok { if !ok {
// not storage system command // not storage system command

View File

@ -54,7 +54,7 @@ func TestWatch(t *testing.T) {
} }
} }
// BenchmarkWatch creates 10K watchers watch at /foo/[paht] each time. // BenchmarkWatch creates 10K watchers watch at /foo/[path] each time.
// Path is randomly chosen with max depth 10. // Path is randomly chosen with max depth 10.
// It should take less than 15ms to wake up 10K watchers. // It should take less than 15ms to wake up 10K watchers.
func BenchmarkWatch(b *testing.B) { func BenchmarkWatch(b *testing.B) {

17
test.go
View File

@ -59,10 +59,10 @@ func createCluster(size int, procAttr *os.ProcAttr) ([][]string, []*os.Process,
argGroup := make([][]string, size) argGroup := make([][]string, size)
for i := 0; i < size; i++ { for i := 0; i < size; i++ {
if i == 0 { if i == 0 {
argGroup[i] = []string{"etcd", "-d=/tmp/node1"} argGroup[i] = []string{"etcd", "-h=127.0.0.1", "-d=/tmp/node1"}
} else { } else {
strI := strconv.Itoa(i + 1) strI := strconv.Itoa(i + 1)
argGroup[i] = []string{"etcd", "-c=400" + strI, "-s=700" + strI, "-d=/tmp/node" + strI, "-C=127.0.0.1:7001"} argGroup[i] = []string{"etcd", "-h=127.0.0.1", "-c=400" + strI, "-s=700" + strI, "-d=/tmp/node" + strI, "-C=127.0.0.1:7001"}
} }
} }
@ -70,13 +70,18 @@ func createCluster(size int, procAttr *os.ProcAttr) ([][]string, []*os.Process,
for i, _ := range etcds { for i, _ := range etcds {
var err error var err error
etcds[i], err = os.StartProcess("etcd", append(argGroup[i], "-i"), procAttr) etcds[i], err = os.StartProcess("etcd", append(argGroup[i], "-f"), procAttr)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
// we only add machine one to the cluster list
// thus we need to make sure other node start after the first one has done set up // TODOBP: Change this sleep to wait until the master is up.
time.Sleep(time.Millisecond * 200) // The problem is that if the master isn't up then the children
// have to retry. This retry can take upwards of 15 seconds
// which slows tests way down and some of them fail.
if i == 0 {
time.Sleep(time.Second)
}
} }
return argGroup, etcds, nil return argGroup, etcds, nil

View File

@ -133,6 +133,11 @@ type Tokenizer struct {
// subsequent Next calls would return an ErrorToken. // subsequent Next calls would return an ErrorToken.
// err is never reset. Once it becomes non-nil, it stays non-nil. // err is never reset. Once it becomes non-nil, it stays non-nil.
err error err error
// readErr is the error returned by the io.Reader r. It is separate from
// err because it is valid for an io.Reader to return (n int, err1 error)
// such that n > 0 && err1 != nil, and callers should always process the
// n > 0 bytes before considering the error err1.
readErr error
// buf[raw.start:raw.end] holds the raw bytes of the current token. // buf[raw.start:raw.end] holds the raw bytes of the current token.
// buf[raw.end:] is buffered input that will yield future tokens. // buf[raw.end:] is buffered input that will yield future tokens.
raw span raw span
@ -222,7 +227,12 @@ func (z *Tokenizer) Err() error {
// Pre-condition: z.err == nil. // Pre-condition: z.err == nil.
func (z *Tokenizer) readByte() byte { func (z *Tokenizer) readByte() byte {
if z.raw.end >= len(z.buf) { if z.raw.end >= len(z.buf) {
// Our buffer is exhausted and we have to read from z.r. // Our buffer is exhausted and we have to read from z.r. Check if the
// previous read resulted in an error.
if z.readErr != nil {
z.err = z.readErr
return 0
}
// We copy z.buf[z.raw.start:z.raw.end] to the beginning of z.buf. If the length // We copy z.buf[z.raw.start:z.raw.end] to the beginning of z.buf. If the length
// z.raw.end - z.raw.start is more than half the capacity of z.buf, then we // z.raw.end - z.raw.start is more than half the capacity of z.buf, then we
// allocate a new buffer before the copy. // allocate a new buffer before the copy.
@ -253,9 +263,10 @@ func (z *Tokenizer) readByte() byte {
z.raw.start, z.raw.end, z.buf = 0, d, buf1[:d] z.raw.start, z.raw.end, z.buf = 0, d, buf1[:d]
// Now that we have copied the live bytes to the start of the buffer, // Now that we have copied the live bytes to the start of the buffer,
// we read from z.r into the remainder. // we read from z.r into the remainder.
n, err := z.r.Read(buf1[d:cap(buf1)]) var n int
if err != nil { n, z.readErr = readAtLeastOneByte(z.r, buf1[d:cap(buf1)])
z.err = err if n == 0 {
z.err = z.readErr
return 0 return 0
} }
z.buf = buf1[:d+n] z.buf = buf1[:d+n]
@ -265,6 +276,19 @@ func (z *Tokenizer) readByte() byte {
return x return x
} }
// readAtLeastOneByte wraps an io.Reader so that reading cannot return (0, nil).
// It returns io.ErrNoProgress if the underlying r.Read method returns (0, nil)
// too many times in succession.
func readAtLeastOneByte(r io.Reader, b []byte) (int, error) {
for i := 0; i < 100; i++ {
n, err := r.Read(b)
if n != 0 || err != nil {
return n, err
}
}
return 0, io.ErrNoProgress
}
// skipWhiteSpace skips past any white space. // skipWhiteSpace skips past any white space.
func (z *Tokenizer) skipWhiteSpace() { func (z *Tokenizer) skipWhiteSpace() {
if z.err != nil { if z.err != nil {

View File

@ -8,6 +8,7 @@ import (
"bytes" "bytes"
"io" "io"
"io/ioutil" "io/ioutil"
"reflect"
"runtime" "runtime"
"strings" "strings"
"testing" "testing"
@ -531,6 +532,85 @@ func TestConvertNewlines(t *testing.T) {
} }
} }
func TestReaderEdgeCases(t *testing.T) {
const s = "<p>An io.Reader can return (0, nil) or (n, io.EOF).</p>"
testCases := []io.Reader{
&zeroOneByteReader{s: s},
&eofStringsReader{s: s},
&stuckReader{},
}
for i, tc := range testCases {
got := []TokenType{}
z := NewTokenizer(tc)
for {
tt := z.Next()
if tt == ErrorToken {
break
}
got = append(got, tt)
}
if err := z.Err(); err != nil && err != io.EOF {
if err != io.ErrNoProgress {
t.Errorf("i=%d: %v", i, err)
}
continue
}
want := []TokenType{
StartTagToken,
TextToken,
EndTagToken,
}
if !reflect.DeepEqual(got, want) {
t.Errorf("i=%d: got %v, want %v", i, got, want)
continue
}
}
}
// zeroOneByteReader is like a strings.Reader that alternates between
// returning 0 bytes and 1 byte at a time.
type zeroOneByteReader struct {
s string
n int
}
func (r *zeroOneByteReader) Read(p []byte) (int, error) {
if len(p) == 0 {
return 0, nil
}
if len(r.s) == 0 {
return 0, io.EOF
}
r.n++
if r.n%2 != 0 {
return 0, nil
}
p[0], r.s = r.s[0], r.s[1:]
return 1, nil
}
// eofStringsReader is like a strings.Reader but can return an (n, err) where
// n > 0 && err != nil.
type eofStringsReader struct {
s string
}
func (r *eofStringsReader) Read(p []byte) (int, error) {
n := copy(p, r.s)
r.s = r.s[n:]
if r.s != "" {
return n, nil
}
return n, io.EOF
}
// stuckReader is an io.Reader that always returns no data and no error.
type stuckReader struct{}
func (*stuckReader) Read(p []byte) (int, error) {
return 0, nil
}
const ( const (
rawLevel = iota rawLevel = iota
lowLevel lowLevel

9
third_party/deps vendored Executable file
View File

@ -0,0 +1,9 @@
packages="
github.com/coreos/go-raft
github.com/coreos/go-etcd
github.com/ccding/go-logging
github.com/ccding/go-config-reader
bitbucket.org/kardianos/osext
code.google.com/p/go.net
code.google.com/p/goprotobuf
"

View File

@ -31,6 +31,7 @@ func Read(filename string) (map[string]string, error) {
if err != nil { if err != nil {
return res, err return res, err
} }
defer in.Close()
scanner := bufio.NewScanner(in) scanner := bufio.NewScanner(in)
line := "" line := ""
section := "" section := ""
@ -60,7 +61,6 @@ func Read(filename string) (map[string]string, error) {
res[section+key] = value res[section+key] = value
line = "" line = ""
} }
in.Close()
return res, nil return res, nil
} }

View File

@ -25,4 +25,6 @@ func main() {
res, err := config.Read("example.conf") res, err := config.Read("example.conf")
fmt.Println(err) fmt.Println(err)
fmt.Println(res) fmt.Println(res)
fmt.Println(res["test.a"])
fmt.Println(res["dd"])
} }

View File

@ -50,25 +50,11 @@ func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*stor
var resp *http.Response var resp *http.Response
var err error var err error
if sinceIndex == 0 { if stop != nil {
// Get request if no index is given
resp, err = c.sendRequest("GET", path.Join("watch", key), "")
if err != nil {
return nil, err
}
} else {
// Post
v := url.Values{}
v.Set("index", fmt.Sprintf("%v", sinceIndex))
ch := make(chan respAndErr) ch := make(chan respAndErr)
if stop != nil {
go func() { go func() {
resp, err = c.sendRequest("POST", path.Join("watch", key), v.Encode()) resp, err = c.sendWatchRequest(key, sinceIndex)
ch <- respAndErr{resp, err} ch <- respAndErr{resp, err}
}() }()
@ -83,15 +69,13 @@ func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*stor
resp, err = nil, errors.New("User stoped watch") resp, err = nil, errors.New("User stoped watch")
} }
} else { } else {
resp, err = c.sendRequest("POST", path.Join("watch", key), v.Encode()) resp, err = c.sendWatchRequest(key, sinceIndex)
} }
if err != nil { if err != nil {
return nil, err return nil, err
} }
}
b, err := ioutil.ReadAll(resp.Body) b, err := ioutil.ReadAll(resp.Body)
resp.Body.Close() resp.Body.Close()
@ -115,3 +99,16 @@ func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*stor
return &result, nil return &result, nil
} }
func (c *Client) sendWatchRequest(key string, sinceIndex uint64) (*http.Response, error) {
if sinceIndex == 0 {
resp, err := c.sendRequest("GET", path.Join("watch", key), "")
return resp, err
} else {
v := url.Values{}
v.Set("index", fmt.Sprintf("%v", sinceIndex))
resp, err := c.sendRequest("POST", path.Join("watch", key), v.Encode())
return resp, err
}
}

View File

@ -14,12 +14,12 @@ func main() {
ch := make(chan bool, 10) ch := make(chan bool, 10)
// set up a lock // set up a lock
c := etcd.CreateClient() c := etcd.NewClient()
c.Set("lock", "unlock", 0) c.Set("lock", "unlock", 0)
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
go t(i, ch, etcd.CreateClient()) go t(i, ch, etcd.NewClient())
} }
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {

View File

@ -11,14 +11,14 @@ var count = 0
func main() { func main() {
ch := make(chan bool, 10) ch := make(chan bool, 10)
// set up a lock // set up a lock
for i:=0; i < 1000; i++ { for i:=0; i < 100; i++ {
go t(i, ch, etcd.CreateClient()) go t(i, ch, etcd.NewClient())
} }
start := time.Now() start := time.Now()
for i:=0; i< 1000; i++ { for i:=0; i< 100; i++ {
<-ch <-ch
} }
fmt.Println(time.Now().Sub(start), ": ", 1000 * 50, "commands") fmt.Println(time.Now().Sub(start), ": ", 100 * 50, "commands")
} }
func t(num int, ch chan bool, c *etcd.Client) { func t(num int, ch chan bool, c *etcd.Client) {

10
third_party/update vendored
View File

@ -1,14 +1,6 @@
#!/bin/sh #!/bin/sh
packages=" . ./deps
github.com/coreos/go-raft
github.com/coreos/go-etcd
github.com/ccding/go-logging
github.com/ccding/go-config-reader
bitbucket.org/kardianos/osext
code.google.com/p/go.net
code.google.com/p/goprotobuf
"
export GOPATH=${PWD} export GOPATH=${PWD}