Compare commits

...

11 Commits

Author SHA1 Message Date
05b564a394 *: bump to v2.2.3 2015-12-30 13:41:16 -08:00
cb779b2305 etcdctl: fix syncWithPeerAPI by breaking the loop when there is no error 2015-12-30 11:24:27 -08:00
22c3208fb3 etcdserver: always check if the data dir is writable before starting etcd 2015-12-30 10:22:52 -08:00
e44372e430 etcdsever: avoid creating member dir before finishing validate bootstrap
This commit fixes the issue of creating member dir before validating
the configuration. When member dir exists, it indicates the local etcd
process is a valid etcd member. So we should only create member dir
after we finish configuration validation, joining validation or
discovery validation.
2015-12-30 10:19:51 -08:00
05a90bc1e5 etcdmain: fix incomplete proxy config file
etcd might generate incomplete proxy config file after a power failure.
It is because we use ioutil.WriteFile. And iotuile.WriteFile does
not call Sync before closing the file.
2015-12-22 12:30:39 -08:00
6751727809 etcdctl: support basic operations with etcd 0.4.
For CoreOS users, they will get a updated version of etcdctl without updating
the etcd server version. And the users cannot really control this behavior.
We do not want to suddenly break them without enough communication.

So we still want the most basic opeartions like get, set, watch of etcdctl2 work
with etcd 0.4. This patches solve the incompability issue.
2015-12-22 12:29:16 -08:00
916106c3a2 client: support reset Endpoints.
ResetEndpoints is useful when the there is a scheduled cluster
changes or when manually manage the cluster without auto-sync
enabled.
2015-12-22 12:25:59 -08:00
e0c7768f94 store: fix data race when modify event in watchHub.
The event got from watchHub should be considered as readonly.
To modify it, we first need to get a clone of it or there might
be a data race.
2015-12-14 14:08:39 -08:00
0fb2d5d4d3 client: fix goroutine leak in unreleased context
If headerTimeout is not zero then two context are created but only one is released.
cancelCtx in this case is never released which leads to goroutine leak inside it.
2015-12-14 13:58:43 -08:00
fc61fc7c7a etcdctl: cluster health exit with non-zero when cluster is unhealthy 2015-12-14 13:58:10 -08:00
09b81bad15 *: bump to v2.2.2+git 2015-11-19 14:24:59 -08:00
11 changed files with 148 additions and 28 deletions

View File

@ -162,6 +162,11 @@ type Client interface {
// this may differ from the initial Endpoints provided in the Config.
Endpoints() []string
// SetEndpoints sets the set of API endpoints used by Client to resolve
// HTTP requests. If the given endpoints are not valid, an error will be
// returned
SetEndpoints(eps []string) error
httpClient
}
@ -176,7 +181,7 @@ func New(cfg Config) (Client, error) {
password: cfg.Password,
}
}
if err := c.reset(cfg.Endpoints); err != nil {
if err := c.SetEndpoints(cfg.Endpoints); err != nil {
return nil, err
}
return c, nil
@ -219,7 +224,7 @@ type httpClusterClient struct {
rand *rand.Rand
}
func (c *httpClusterClient) reset(eps []string) error {
func (c *httpClusterClient) SetEndpoints(eps []string) error {
if len(eps) == 0 {
return ErrNoEndpoints
}
@ -341,7 +346,7 @@ func (c *httpClusterClient) Sync(ctx context.Context) error {
return nil
}
return c.reset(eps)
return c.SetEndpoints(eps)
}
func (c *httpClusterClient) AutoSync(ctx context.Context, interval time.Duration) error {
@ -378,9 +383,12 @@ func (c *simpleHTTPClient) Do(ctx context.Context, act httpAction) (*http.Respon
return nil, nil, err
}
hctx, hcancel := context.WithCancel(ctx)
var hctx context.Context
var hcancel context.CancelFunc
if c.headerTimeout > 0 {
hctx, hcancel = context.WithTimeout(ctx, c.headerTimeout)
} else {
hctx, hcancel = context.WithCancel(ctx)
}
defer hcancel()

View File

@ -705,7 +705,7 @@ func TestHTTPClusterClientSync(t *testing.T) {
clientFactory: cf,
rand: rand.New(rand.NewSource(0)),
}
err := hc.reset([]string{"http://127.0.0.1:2379"})
err := hc.SetEndpoints([]string{"http://127.0.0.1:2379"})
if err != nil {
t.Fatalf("unexpected error during setup: %#v", err)
}
@ -728,7 +728,7 @@ func TestHTTPClusterClientSync(t *testing.T) {
t.Fatalf("incorrect endpoints post-Sync: want=%#v got=%#v", want, got)
}
err = hc.reset([]string{"http://127.0.0.1:4009"})
err = hc.SetEndpoints([]string{"http://127.0.0.1:4009"})
if err != nil {
t.Fatalf("unexpected error during reset: %#v", err)
}
@ -749,7 +749,7 @@ func TestHTTPClusterClientSyncFail(t *testing.T) {
clientFactory: cf,
rand: rand.New(rand.NewSource(0)),
}
err := hc.reset([]string{"http://127.0.0.1:2379"})
err := hc.SetEndpoints([]string{"http://127.0.0.1:2379"})
if err != nil {
t.Fatalf("unexpected error during setup: %#v", err)
}
@ -783,7 +783,7 @@ func TestHTTPClusterClientAutoSyncCancelContext(t *testing.T) {
clientFactory: cf,
rand: rand.New(rand.NewSource(0)),
}
err := hc.reset([]string{"http://127.0.0.1:2379"})
err := hc.SetEndpoints([]string{"http://127.0.0.1:2379"})
if err != nil {
t.Fatalf("unexpected error during setup: %#v", err)
}
@ -805,7 +805,7 @@ func TestHTTPClusterClientAutoSyncFail(t *testing.T) {
clientFactory: cf,
rand: rand.New(rand.NewSource(0)),
}
err := hc.reset([]string{"http://127.0.0.1:2379"})
err := hc.SetEndpoints([]string{"http://127.0.0.1:2379"})
if err != nil {
t.Fatalf("unexpected error during setup: %#v", err)
}
@ -838,7 +838,7 @@ func TestHTTPClusterClientSyncPinEndpoint(t *testing.T) {
clientFactory: cf,
rand: rand.New(rand.NewSource(0)),
}
err := hc.reset([]string{"http://127.0.0.1:4003", "http://127.0.0.1:2379", "http://127.0.0.1:4001", "http://127.0.0.1:4002"})
err := hc.SetEndpoints([]string{"http://127.0.0.1:4003", "http://127.0.0.1:2379", "http://127.0.0.1:4001", "http://127.0.0.1:4002"})
if err != nil {
t.Fatalf("unexpected error during setup: %#v", err)
}
@ -867,7 +867,7 @@ func TestHTTPClusterClientResetFail(t *testing.T) {
for i, tt := range tests {
hc := &httpClusterClient{rand: rand.New(rand.NewSource(0))}
err := hc.reset(tt)
err := hc.SetEndpoints(tt)
if err == nil {
t.Errorf("#%d: expected non-nil error", i)
}
@ -879,7 +879,7 @@ func TestHTTPClusterClientResetPinRandom(t *testing.T) {
pinNum := 0
for i := 0; i < round; i++ {
hc := &httpClusterClient{rand: rand.New(rand.NewSource(int64(i)))}
err := hc.reset([]string{"http://127.0.0.1:4001", "http://127.0.0.1:4002", "http://127.0.0.1:4003"})
err := hc.SetEndpoints([]string{"http://127.0.0.1:4001", "http://127.0.0.1:4002", "http://127.0.0.1:4003"})
if err != nil {
t.Fatalf("#%d: reset error (%v)", i, err)
}

View File

@ -108,8 +108,13 @@ func handleClusterHealth(c *cli.Context) {
}
if !forever {
break
if health {
os.Exit(ExitSuccess)
} else {
os.Exit(ExitClusterNotHealthy)
}
}
fmt.Printf("\nnext check after 10 second...\n\n")
time.Sleep(10 * time.Second)
}

View File

@ -27,6 +27,7 @@ const (
ExitBadConnection
ExitBadAuth
ExitServerError
ExitClusterNotHealthy
)
func handleError(code int, err error) {

View File

@ -203,9 +203,19 @@ func mustNewClient(c *cli.Context) client.Client {
if err == client.ErrNoEndpoints {
fmt.Fprintf(os.Stderr, "etcd cluster has no published client endpoints.\n")
fmt.Fprintf(os.Stderr, "Try '--no-sync' if you want to access non-published client endpoints(%s).\n", strings.Join(hc.Endpoints(), ","))
handleError(ExitServerError, err)
}
// fail-back to try sync cluster with peer API. this is for making etcdctl work with etcd 0.4.x.
// TODO: remove this when we deprecate the support for etcd 0.4.
eps, serr := syncWithPeerAPI(c, ctx, hc.Endpoints())
if serr != nil {
handleError(ExitServerError, serr)
}
err = hc.SetEndpoints(eps)
if err != nil {
handleError(ExitServerError, err)
}
handleError(ExitServerError, err)
os.Exit(1)
}
if debug {
fmt.Fprintf(os.Stderr, "got endpoints(%s) after sync\n", strings.Join(hc.Endpoints(), ","))
@ -267,3 +277,44 @@ func newClient(c *cli.Context) (client.Client, error) {
func contextWithTotalTimeout(c *cli.Context) (context.Context, context.CancelFunc) {
return context.WithTimeout(context.Background(), c.GlobalDuration("total-timeout"))
}
// syncWithPeerAPI syncs cluster with peer API defined at
// https://github.com/coreos/etcd/blob/v0.4.9/server/server.go#L311.
// This exists for backward compatibility with etcd 0.4.x.
func syncWithPeerAPI(c *cli.Context, ctx context.Context, knownPeers []string) ([]string, error) {
tr, err := getTransport(c)
if err != nil {
return nil, err
}
var (
body []byte
resp *http.Response
)
for _, p := range knownPeers {
var req *http.Request
req, err = http.NewRequest("GET", p+"/v2/peers", nil)
if err != nil {
continue
}
resp, err = tr.RoundTrip(req)
if err != nil {
continue
}
if resp.StatusCode != http.StatusOK {
resp.Body.Close()
continue
}
body, err = ioutil.ReadAll(resp.Body)
resp.Body.Close()
if err == nil {
break
}
}
if err != nil {
return nil, err
}
// Parse the peers API format: https://github.com/coreos/etcd/blob/v0.4.9/server/server.go#L311
return strings.Split(string(body), ", "), nil
}

View File

@ -40,6 +40,7 @@ import (
"github.com/coreos/etcd/etcdserver/etcdserverpb"
"github.com/coreos/etcd/pkg/cors"
"github.com/coreos/etcd/pkg/fileutil"
pkgioutil "github.com/coreos/etcd/pkg/ioutil"
"github.com/coreos/etcd/pkg/osutil"
runtimeutil "github.com/coreos/etcd/pkg/runtime"
"github.com/coreos/etcd/pkg/transport"
@ -409,7 +410,7 @@ func startProxy(cfg *config) error {
return clientURLs
}
err = ioutil.WriteFile(clusterfile+".bak", b, 0600)
err = pkgioutil.WriteAndSyncFile(clusterfile+".bak", b, 0600)
if err != nil {
plog.Warningf("proxy: error on writing urls %s", err)
return clientURLs

View File

@ -174,12 +174,17 @@ type EtcdServer struct {
// configuration is considered static for the lifetime of the EtcdServer.
func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
st := store.New(StoreClusterPrefix, StoreKeysPrefix)
var w *wal.WAL
var n raft.Node
var s *raft.MemoryStorage
var id types.ID
var cl *cluster
if terr := fileutil.TouchDirAll(cfg.DataDir); terr != nil {
return nil, fmt.Errorf("cannot access data directory: %v", terr)
}
// Run the migrations.
dataVer, err := version.DetectDataDir(cfg.DataDir)
if err != nil {
@ -189,11 +194,6 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
return nil, err
}
err = os.MkdirAll(cfg.MemberDir(), privateDirMode)
if err != nil && err != os.ErrExist {
return nil, err
}
haveWAL := wal.Exist(cfg.WALDir())
ss := snap.New(cfg.SnapDir())
@ -255,10 +255,6 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
cfg.PrintWithInitial()
id, n, s, w = startNode(cfg, cl, cl.MemberIDs())
case haveWAL:
if err := fileutil.IsDirWriteable(cfg.DataDir); err != nil {
return nil, fmt.Errorf("cannot write to data directory: %v", err)
}
if err := fileutil.IsDirWriteable(cfg.MemberDir()); err != nil {
return nil, fmt.Errorf("cannot write to member directory: %v", err)
}
@ -295,6 +291,10 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
return nil, fmt.Errorf("unsupported bootstrap config")
}
if terr := fileutil.TouchDirAll(cfg.MemberDir()); terr != nil {
return nil, fmt.Errorf("cannot access member directory: %v", terr)
}
sstats := &stats.ServerStats{
Name: cfg.Name,
ID: id.String(),

View File

@ -25,6 +25,8 @@ import (
const (
privateFileMode = 0600
// owner can make/remove files inside the directory
privateDirMode = 0700
)
var (
@ -55,3 +57,13 @@ func ReadDir(dirpath string) ([]string, error) {
sort.Strings(names)
return names, nil
}
// TouchDirAll is simliar to os.MkdirAll. It creates directories with 0700 permission if any directory
// does not exists. TouchDirAll also ensures the given directory is writable.
func TouchDirAll(dir string) error {
err := os.MkdirAll(dir, privateDirMode)
if err != nil && err != os.ErrExist {
return err
}
return IsDirWriteable(dir)
}

41
pkg/ioutil/util.go Normal file
View File

@ -0,0 +1,41 @@
// Copyright 2015 CoreOS, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package ioutil
import (
"io"
"os"
)
// WriteAndSyncFile behaviors just like ioutil.WriteFile in standard library
// but calls Sync before closing the file. WriteAndSyncFile guarantees the data
// is synced if there is no error returned.
func WriteAndSyncFile(filename string, data []byte, perm os.FileMode) error {
f, err := os.OpenFile(filename, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, perm)
if err != nil {
return err
}
n, err := f.Write(data)
if err == nil && n < len(data) {
err = io.ErrShortWrite
}
if err == nil {
err = f.Sync()
}
if err1 := f.Close(); err == nil {
err = err1
}
return err
}

View File

@ -78,8 +78,9 @@ func (wh *watcherHub) watch(key string, recursive, stream bool, index, storeInde
defer wh.mutex.Unlock()
// If the event exists in the known history, append the EtcdIndex and return immediately
if event != nil {
event.EtcdIndex = storeIndex
w.eventChan <- event
ne := event.Clone()
ne.EtcdIndex = storeIndex
w.eventChan <- ne
return w, nil
}

View File

@ -27,7 +27,7 @@ import (
var (
// MinClusterVersion is the min cluster version this etcd binary is compatible with.
MinClusterVersion = "2.1.0"
Version = "2.2.2"
Version = "2.2.3"
// Git SHA Value will be set during build
GitSHA = "Not provided (use ./build instead of go build)"