Compare commits
11 Commits
Author | SHA1 | Date | |
---|---|---|---|
c99d0d4b25 | |||
d78216f528 | |||
c05c027a24 | |||
3fd64f913a | |||
f935290bbc | |||
ca91f898a2 | |||
fcbada7798 | |||
fad9bdc3e1 | |||
198ccb8b7b | |||
dc5d5c6ac8 | |||
f771eaca47 |
@ -603,6 +603,11 @@ func (as *authStore) isOpPermitted(userName string, key, rangeEnd []byte, permTy
|
|||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// root role should have permission on all ranges
|
||||||
|
if hasRootRole(user) {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
if as.isRangeOpPermitted(tx, userName, key, rangeEnd, permTyp) {
|
if as.isRangeOpPermitted(tx, userName, key, rangeEnd, permTyp) {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
@ -32,35 +32,63 @@ func ExampleAuth() {
|
|||||||
}
|
}
|
||||||
defer cli.Close()
|
defer cli.Close()
|
||||||
|
|
||||||
authapi := clientv3.NewAuth(cli)
|
if _, err = cli.RoleAdd(context.TODO(), "root"); err != nil {
|
||||||
|
log.Fatal(err)
|
||||||
if _, err = authapi.RoleAdd(context.TODO(), "root"); err != nil {
|
}
|
||||||
|
if _, err = cli.UserAdd(context.TODO(), "root", "123"); err != nil {
|
||||||
|
log.Fatal(err)
|
||||||
|
}
|
||||||
|
if _, err = cli.UserGrantRole(context.TODO(), "root", "root"); err != nil {
|
||||||
log.Fatal(err)
|
log.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
if _, err = authapi.RoleGrantPermission(
|
if _, err = cli.RoleAdd(context.TODO(), "r"); err != nil {
|
||||||
|
log.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
if _, err = cli.RoleGrantPermission(
|
||||||
context.TODO(),
|
context.TODO(),
|
||||||
"root", // role name
|
"r", // role name
|
||||||
"foo", // key
|
"foo", // key
|
||||||
"zoo", // range end
|
"zoo", // range end
|
||||||
clientv3.PermissionType(clientv3.PermReadWrite),
|
clientv3.PermissionType(clientv3.PermReadWrite),
|
||||||
); err != nil {
|
); err != nil {
|
||||||
log.Fatal(err)
|
log.Fatal(err)
|
||||||
}
|
}
|
||||||
|
if _, err = cli.UserAdd(context.TODO(), "u", "123"); err != nil {
|
||||||
if _, err = authapi.UserAdd(context.TODO(), "root", "123"); err != nil {
|
|
||||||
log.Fatal(err)
|
log.Fatal(err)
|
||||||
}
|
}
|
||||||
|
if _, err = cli.UserGrantRole(context.TODO(), "u", "r"); err != nil {
|
||||||
if _, err = authapi.UserGrantRole(context.TODO(), "root", "root"); err != nil {
|
|
||||||
log.Fatal(err)
|
log.Fatal(err)
|
||||||
}
|
}
|
||||||
|
if _, err = cli.AuthEnable(context.TODO()); err != nil {
|
||||||
if _, err = authapi.AuthEnable(context.TODO()); err != nil {
|
|
||||||
log.Fatal(err)
|
log.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
cliAuth, err := clientv3.New(clientv3.Config{
|
cliAuth, err := clientv3.New(clientv3.Config{
|
||||||
|
Endpoints: endpoints,
|
||||||
|
DialTimeout: dialTimeout,
|
||||||
|
Username: "u",
|
||||||
|
Password: "123",
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
log.Fatal(err)
|
||||||
|
}
|
||||||
|
defer cliAuth.Close()
|
||||||
|
|
||||||
|
if _, err = cliAuth.Put(context.TODO(), "foo1", "bar"); err != nil {
|
||||||
|
log.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
_, err = cliAuth.Txn(context.TODO()).
|
||||||
|
If(clientv3.Compare(clientv3.Value("zoo1"), ">", "abc")).
|
||||||
|
Then(clientv3.OpPut("zoo1", "XYZ")).
|
||||||
|
Else(clientv3.OpPut("zoo1", "ABC")).
|
||||||
|
Commit()
|
||||||
|
fmt.Println(err)
|
||||||
|
|
||||||
|
// now check the permission with the root account
|
||||||
|
rootCli, err := clientv3.New(clientv3.Config{
|
||||||
Endpoints: endpoints,
|
Endpoints: endpoints,
|
||||||
DialTimeout: dialTimeout,
|
DialTimeout: dialTimeout,
|
||||||
Username: "root",
|
Username: "root",
|
||||||
@ -69,31 +97,17 @@ func ExampleAuth() {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatal(err)
|
log.Fatal(err)
|
||||||
}
|
}
|
||||||
defer cliAuth.Close()
|
defer rootCli.Close()
|
||||||
|
|
||||||
kv := clientv3.NewKV(cliAuth)
|
resp, err := rootCli.RoleGet(context.TODO(), "r")
|
||||||
if _, err = kv.Put(context.TODO(), "foo1", "bar"); err != nil {
|
|
||||||
log.Fatal(err)
|
|
||||||
}
|
|
||||||
|
|
||||||
_, err = kv.Txn(context.TODO()).
|
|
||||||
If(clientv3.Compare(clientv3.Value("zoo1"), ">", "abc")).
|
|
||||||
Then(clientv3.OpPut("zoo1", "XYZ")).
|
|
||||||
Else(clientv3.OpPut("zoo1", "ABC")).
|
|
||||||
Commit()
|
|
||||||
fmt.Println(err)
|
|
||||||
|
|
||||||
// now check the permission
|
|
||||||
authapi2 := clientv3.NewAuth(cliAuth)
|
|
||||||
resp, err := authapi2.RoleGet(context.TODO(), "root")
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatal(err)
|
log.Fatal(err)
|
||||||
}
|
}
|
||||||
fmt.Printf("root user permission: key %q, range end %q\n", resp.Perm[0].Key, resp.Perm[0].RangeEnd)
|
fmt.Printf("user u permission: key %q, range end %q\n", resp.Perm[0].Key, resp.Perm[0].RangeEnd)
|
||||||
|
|
||||||
if _, err = authapi2.AuthDisable(context.TODO()); err != nil {
|
if _, err = rootCli.AuthDisable(context.TODO()); err != nil {
|
||||||
log.Fatal(err)
|
log.Fatal(err)
|
||||||
}
|
}
|
||||||
// Output: etcdserver: permission denied
|
// Output: etcdserver: permission denied
|
||||||
// root user permission: key "foo", range end "zoo"
|
// user u permission: key "foo", range end "zoo"
|
||||||
}
|
}
|
||||||
|
@ -75,11 +75,11 @@ func authCredWriteKeyTest(cx ctlCtx) {
|
|||||||
cx.user, cx.pass = "root", "root"
|
cx.user, cx.pass = "root", "root"
|
||||||
authSetupTestUser(cx)
|
authSetupTestUser(cx)
|
||||||
|
|
||||||
// confirm root role doesn't grant access to all keys
|
// confirm root role can access to all keys
|
||||||
if err := ctlV3PutFailPerm(cx, "foo", "bar"); err != nil {
|
if err := ctlV3Put(cx, "foo", "bar", ""); err != nil {
|
||||||
cx.t.Fatal(err)
|
cx.t.Fatal(err)
|
||||||
}
|
}
|
||||||
if err := ctlV3GetFailPerm(cx, "foo"); err != nil {
|
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "bar"}}...); err != nil {
|
||||||
cx.t.Fatal(err)
|
cx.t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -90,17 +90,17 @@ func authCredWriteKeyTest(cx ctlCtx) {
|
|||||||
}
|
}
|
||||||
// confirm put failed
|
// confirm put failed
|
||||||
cx.user, cx.pass = "test-user", "pass"
|
cx.user, cx.pass = "test-user", "pass"
|
||||||
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "a"}}...); err != nil {
|
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "bar"}}...); err != nil {
|
||||||
cx.t.Fatal(err)
|
cx.t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// try good user
|
// try good user
|
||||||
cx.user, cx.pass = "test-user", "pass"
|
cx.user, cx.pass = "test-user", "pass"
|
||||||
if err := ctlV3Put(cx, "foo", "bar", ""); err != nil {
|
if err := ctlV3Put(cx, "foo", "bar2", ""); err != nil {
|
||||||
cx.t.Fatal(err)
|
cx.t.Fatal(err)
|
||||||
}
|
}
|
||||||
// confirm put succeeded
|
// confirm put succeeded
|
||||||
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "bar"}}...); err != nil {
|
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "bar2"}}...); err != nil {
|
||||||
cx.t.Fatal(err)
|
cx.t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -111,7 +111,7 @@ func authCredWriteKeyTest(cx ctlCtx) {
|
|||||||
}
|
}
|
||||||
// confirm put failed
|
// confirm put failed
|
||||||
cx.user, cx.pass = "test-user", "pass"
|
cx.user, cx.pass = "test-user", "pass"
|
||||||
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "bar"}}...); err != nil {
|
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "bar2"}}...); err != nil {
|
||||||
cx.t.Fatal(err)
|
cx.t.Fatal(err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -282,10 +282,6 @@ func ctlV3PutFailPerm(cx ctlCtx, key, val string) error {
|
|||||||
return spawnWithExpect(append(cx.PrefixArgs(), "put", key, val), "permission denied")
|
return spawnWithExpect(append(cx.PrefixArgs(), "put", key, val), "permission denied")
|
||||||
}
|
}
|
||||||
|
|
||||||
func ctlV3GetFailPerm(cx ctlCtx, key string) error {
|
|
||||||
return spawnWithExpect(append(cx.PrefixArgs(), "get", key), "permission denied")
|
|
||||||
}
|
|
||||||
|
|
||||||
func authSetupTestUser(cx ctlCtx) {
|
func authSetupTestUser(cx ctlCtx) {
|
||||||
if err := ctlV3User(cx, []string{"add", "test-user", "--interactive=false"}, "User test-user created", []string{"pass"}); err != nil {
|
if err := ctlV3User(cx, []string{"add", "test-user", "--interactive=false"}, "User test-user created", []string{"pass"}); err != nil {
|
||||||
cx.t.Fatal(err)
|
cx.t.Fatal(err)
|
||||||
|
@ -220,7 +220,7 @@ func writeKeys(w io.Writer, n *store.NodeExtern) uint64 {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
ExitWithError(ExitError, err)
|
ExitWithError(ExitError, err)
|
||||||
}
|
}
|
||||||
fmt.Fprintf(w, string(b))
|
fmt.Fprint(w, string(b))
|
||||||
for _, nn := range nodes {
|
for _, nn := range nodes {
|
||||||
max := writeKeys(w, nn)
|
max := writeKeys(w, nn)
|
||||||
if max > maxIndex {
|
if max > maxIndex {
|
||||||
|
@ -154,13 +154,13 @@ type Server interface {
|
|||||||
|
|
||||||
// EtcdServer is the production implementation of the Server interface
|
// EtcdServer is the production implementation of the Server interface
|
||||||
type EtcdServer struct {
|
type EtcdServer struct {
|
||||||
// r and inflightSnapshots must be the first elements to keep 64-bit alignment for atomic
|
// inflightSnapshots holds count the number of snapshots currently inflight.
|
||||||
// access to fields
|
inflightSnapshots int64 // must use atomic operations to access; keep 64-bit aligned.
|
||||||
|
appliedIndex uint64 // must use atomic operations to access; keep 64-bit aligned.
|
||||||
// count the number of inflight snapshots.
|
// consistIndex used to hold the offset of current executing entry
|
||||||
// MUST use atomic operation to access this field.
|
// It is initialized to 0 before executing any entry.
|
||||||
inflightSnapshots int64
|
consistIndex consistentIndex // must use atomic operations to access; keep 64-bit aligned.
|
||||||
Cfg *ServerConfig
|
Cfg *ServerConfig
|
||||||
|
|
||||||
readych chan struct{}
|
readych chan struct{}
|
||||||
r raftNode
|
r raftNode
|
||||||
@ -195,10 +195,6 @@ type EtcdServer struct {
|
|||||||
// compactor is used to auto-compact the KV.
|
// compactor is used to auto-compact the KV.
|
||||||
compactor *compactor.Periodic
|
compactor *compactor.Periodic
|
||||||
|
|
||||||
// consistent index used to hold the offset of current executing entry
|
|
||||||
// It is initialized to 0 before executing any entry.
|
|
||||||
consistIndex consistentIndex
|
|
||||||
|
|
||||||
// peerRt used to send requests (version, lease) to peers.
|
// peerRt used to send requests (version, lease) to peers.
|
||||||
peerRt http.RoundTripper
|
peerRt http.RoundTripper
|
||||||
reqIDGen *idutil.Generator
|
reqIDGen *idutil.Generator
|
||||||
@ -212,8 +208,6 @@ type EtcdServer struct {
|
|||||||
// wg is used to wait for the go routines that depends on the server state
|
// wg is used to wait for the go routines that depends on the server state
|
||||||
// to exit when stopping the server.
|
// to exit when stopping the server.
|
||||||
wg sync.WaitGroup
|
wg sync.WaitGroup
|
||||||
|
|
||||||
appliedIndex uint64
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewServer creates a new EtcdServer from the supplied configuration. The
|
// NewServer creates a new EtcdServer from the supplied configuration. The
|
||||||
|
@ -408,6 +408,13 @@ func (s *store) restore() error {
|
|||||||
s.currentRev = rev
|
s.currentRev = rev
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// keys in the range [compacted revision -N, compaction] might all be deleted due to compaction.
|
||||||
|
// the correct revision should be set to compaction revision in the case, not the largest revision
|
||||||
|
// we have seen.
|
||||||
|
if s.currentRev.main < s.compactMainRev {
|
||||||
|
s.currentRev.main = s.compactMainRev
|
||||||
|
}
|
||||||
|
|
||||||
for key, lid := range keyToLease {
|
for key, lid := range keyToLease {
|
||||||
if s.le == nil {
|
if s.le == nil {
|
||||||
panic("no lessor to attach lease")
|
panic("no lessor to attach lease")
|
||||||
|
@ -15,8 +15,10 @@
|
|||||||
package mvcc
|
package mvcc
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"os"
|
||||||
"reflect"
|
"reflect"
|
||||||
"testing"
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/coreos/etcd/lease"
|
"github.com/coreos/etcd/lease"
|
||||||
"github.com/coreos/etcd/mvcc/backend"
|
"github.com/coreos/etcd/mvcc/backend"
|
||||||
@ -93,3 +95,41 @@ func TestScheduleCompaction(t *testing.T) {
|
|||||||
cleanup(s, b, tmpPath)
|
cleanup(s, b, tmpPath)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestCompactAllAndRestore(t *testing.T) {
|
||||||
|
b, tmpPath := backend.NewDefaultTmpBackend()
|
||||||
|
s0 := NewStore(b, &lease.FakeLessor{}, nil)
|
||||||
|
defer os.Remove(tmpPath)
|
||||||
|
|
||||||
|
s0.Put([]byte("foo"), []byte("bar"), lease.NoLease)
|
||||||
|
s0.Put([]byte("foo"), []byte("bar1"), lease.NoLease)
|
||||||
|
s0.Put([]byte("foo"), []byte("bar2"), lease.NoLease)
|
||||||
|
s0.DeleteRange([]byte("foo"), nil)
|
||||||
|
|
||||||
|
rev := s0.Rev()
|
||||||
|
// compact all keys
|
||||||
|
done, err := s0.Compact(rev)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-done:
|
||||||
|
case <-time.After(10 * time.Second):
|
||||||
|
t.Fatal("timeout waiting for compaction to finish")
|
||||||
|
}
|
||||||
|
|
||||||
|
err = s0.Close()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
s1 := NewStore(b, &lease.FakeLessor{}, nil)
|
||||||
|
if s1.Rev() != rev {
|
||||||
|
t.Errorf("rev = %v, want %v", s1.Rev(), rev)
|
||||||
|
}
|
||||||
|
_, err = s1.Range([]byte("foo"), nil, RangeOptions{})
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("unexpect range error %v", err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@ -38,7 +38,7 @@ var (
|
|||||||
// SoftState provides state that is useful for logging and debugging.
|
// SoftState provides state that is useful for logging and debugging.
|
||||||
// The state is volatile and does not need to be persisted to the WAL.
|
// The state is volatile and does not need to be persisted to the WAL.
|
||||||
type SoftState struct {
|
type SoftState struct {
|
||||||
Lead uint64
|
Lead uint64 // must use atomic operations to access; keep 64-bit aligned.
|
||||||
RaftState StateType
|
RaftState StateType
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -183,9 +183,9 @@ func (x *ConfChangeType) UnmarshalJSON(data []byte) error {
|
|||||||
func (ConfChangeType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{2} }
|
func (ConfChangeType) EnumDescriptor() ([]byte, []int) { return fileDescriptorRaft, []int{2} }
|
||||||
|
|
||||||
type Entry struct {
|
type Entry struct {
|
||||||
Type EntryType `protobuf:"varint,1,opt,name=Type,json=type,enum=raftpb.EntryType" json:"Type"`
|
|
||||||
Term uint64 `protobuf:"varint,2,opt,name=Term,json=term" json:"Term"`
|
Term uint64 `protobuf:"varint,2,opt,name=Term,json=term" json:"Term"`
|
||||||
Index uint64 `protobuf:"varint,3,opt,name=Index,json=index" json:"Index"`
|
Index uint64 `protobuf:"varint,3,opt,name=Index,json=index" json:"Index"`
|
||||||
|
Type EntryType `protobuf:"varint,1,opt,name=Type,json=type,enum=raftpb.EntryType" json:"Type"`
|
||||||
Data []byte `protobuf:"bytes,4,opt,name=Data,json=data" json:"Data,omitempty"`
|
Data []byte `protobuf:"bytes,4,opt,name=Data,json=data" json:"Data,omitempty"`
|
||||||
XXX_unrecognized []byte `json:"-"`
|
XXX_unrecognized []byte `json:"-"`
|
||||||
}
|
}
|
||||||
|
@ -15,9 +15,9 @@ enum EntryType {
|
|||||||
}
|
}
|
||||||
|
|
||||||
message Entry {
|
message Entry {
|
||||||
|
optional uint64 Term = 2 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations
|
||||||
|
optional uint64 Index = 3 [(gogoproto.nullable) = false]; // must be 64-bit aligned for atomic operations
|
||||||
optional EntryType Type = 1 [(gogoproto.nullable) = false];
|
optional EntryType Type = 1 [(gogoproto.nullable) = false];
|
||||||
optional uint64 Term = 2 [(gogoproto.nullable) = false];
|
|
||||||
optional uint64 Index = 3 [(gogoproto.nullable) = false];
|
|
||||||
optional bytes Data = 4;
|
optional bytes Data = 4;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -29,7 +29,7 @@ import (
|
|||||||
var (
|
var (
|
||||||
// MinClusterVersion is the min cluster version this etcd binary is compatible with.
|
// MinClusterVersion is the min cluster version this etcd binary is compatible with.
|
||||||
MinClusterVersion = "2.3.0"
|
MinClusterVersion = "2.3.0"
|
||||||
Version = "3.0.12"
|
Version = "3.0.13"
|
||||||
|
|
||||||
// Git SHA Value will be set during build
|
// Git SHA Value will be set during build
|
||||||
GitSHA = "Not provided (use ./build instead of go build)"
|
GitSHA = "Not provided (use ./build instead of go build)"
|
||||||
|
Reference in New Issue
Block a user