mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
clientv3: remove dependency on lease package
This commit is contained in:
parent
44753594ec
commit
a001651bc1
@ -18,7 +18,6 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
|
||||
v3 "github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/lease"
|
||||
)
|
||||
|
||||
// only keep one ephemeral lease per client
|
||||
@ -35,7 +34,7 @@ type clientSessionMgr struct {
|
||||
// Fault-tolerant applications may use sessions to reason about liveness.
|
||||
type Session struct {
|
||||
client *v3.Client
|
||||
id lease.LeaseID
|
||||
id v3.LeaseID
|
||||
|
||||
cancel context.CancelFunc
|
||||
donec <-chan struct{}
|
||||
@ -53,7 +52,7 @@ func NewSession(client *v3.Client) (*Session, error) {
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
id := lease.LeaseID(resp.ID)
|
||||
id := v3.LeaseID(resp.ID)
|
||||
|
||||
ctx, cancel := context.WithCancel(client.Ctx())
|
||||
keepAlive, err := client.KeepAlive(ctx, id)
|
||||
@ -82,7 +81,7 @@ func NewSession(client *v3.Client) (*Session, error) {
|
||||
}
|
||||
|
||||
// Lease is the lease ID for keys bound to the session.
|
||||
func (s *Session) Lease() lease.LeaseID { return s.id }
|
||||
func (s *Session) Lease() v3.LeaseID { return s.id }
|
||||
|
||||
// Done returns a channel that closes when the lease is orphaned, expires, or
|
||||
// is otherwise no longer being refreshed.
|
||||
|
@ -20,7 +20,6 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/lease"
|
||||
)
|
||||
|
||||
func ExampleLease_create() {
|
||||
@ -40,7 +39,7 @@ func ExampleLease_create() {
|
||||
}
|
||||
|
||||
// after 5 seconds, the key 'foo' will be removed
|
||||
_, err = cli.Put(context.TODO(), "foo", "bar", clientv3.WithLease(lease.LeaseID(resp.ID)))
|
||||
_, err = cli.Put(context.TODO(), "foo", "bar", clientv3.WithLease(clientv3.LeaseID(resp.ID)))
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
@ -61,13 +60,13 @@ func ExampleLease_revoke() {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
_, err = cli.Put(context.TODO(), "foo", "bar", clientv3.WithLease(lease.LeaseID(resp.ID)))
|
||||
_, err = cli.Put(context.TODO(), "foo", "bar", clientv3.WithLease(clientv3.LeaseID(resp.ID)))
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// revoking lease expires the key attached to its lease ID
|
||||
_, err = cli.Revoke(context.TODO(), lease.LeaseID(resp.ID))
|
||||
_, err = cli.Revoke(context.TODO(), clientv3.LeaseID(resp.ID))
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
@ -95,13 +94,13 @@ func ExampleLease_keepAlive() {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
_, err = cli.Put(context.TODO(), "foo", "bar", clientv3.WithLease(lease.LeaseID(resp.ID)))
|
||||
_, err = cli.Put(context.TODO(), "foo", "bar", clientv3.WithLease(clientv3.LeaseID(resp.ID)))
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// the key 'foo' will be kept forever
|
||||
_, err = cli.KeepAlive(context.TODO(), lease.LeaseID(resp.ID))
|
||||
_, err = cli.KeepAlive(context.TODO(), clientv3.LeaseID(resp.ID))
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
@ -122,13 +121,13 @@ func ExampleLease_keepAliveOnce() {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
_, err = cli.Put(context.TODO(), "foo", "bar", clientv3.WithLease(lease.LeaseID(resp.ID)))
|
||||
_, err = cli.Put(context.TODO(), "foo", "bar", clientv3.WithLease(clientv3.LeaseID(resp.ID)))
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
||||
// to renew the lease only once
|
||||
_, err = cli.KeepAliveOnce(context.TODO(), lease.LeaseID(resp.ID))
|
||||
_, err = cli.KeepAliveOnce(context.TODO(), clientv3.LeaseID(resp.ID))
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
|
@ -24,7 +24,6 @@ import (
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"github.com/coreos/etcd/integration"
|
||||
"github.com/coreos/etcd/lease"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/storage/storagepb"
|
||||
)
|
||||
@ -48,10 +47,10 @@ func TestKVPut(t *testing.T) {
|
||||
|
||||
tests := []struct {
|
||||
key, val string
|
||||
leaseID lease.LeaseID
|
||||
leaseID clientv3.LeaseID
|
||||
}{
|
||||
{"foo", "bar", lease.NoLease},
|
||||
{"hello", "world", lease.LeaseID(resp.ID)},
|
||||
{"foo", "bar", clientv3.NoLease},
|
||||
{"hello", "world", clientv3.LeaseID(resp.ID)},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
@ -68,7 +67,7 @@ func TestKVPut(t *testing.T) {
|
||||
if !bytes.Equal([]byte(tt.val), resp.Kvs[0].Value) {
|
||||
t.Errorf("#%d: val = %s, want %s", i, tt.val, resp.Kvs[0].Value)
|
||||
}
|
||||
if tt.leaseID != lease.LeaseID(resp.Kvs[0].Lease) {
|
||||
if tt.leaseID != clientv3.LeaseID(resp.Kvs[0].Lease) {
|
||||
t.Errorf("#%d: val = %d, want %d", i, tt.leaseID, resp.Kvs[0].Lease)
|
||||
}
|
||||
}
|
||||
|
@ -22,7 +22,6 @@ import (
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"github.com/coreos/etcd/integration"
|
||||
"github.com/coreos/etcd/lease"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
)
|
||||
|
||||
@ -42,7 +41,7 @@ func TestLeaseCreate(t *testing.T) {
|
||||
t.Errorf("failed to create lease %v", err)
|
||||
}
|
||||
|
||||
_, err = kv.Put(context.TODO(), "foo", "bar", clientv3.WithLease(lease.LeaseID(resp.ID)))
|
||||
_, err = kv.Put(context.TODO(), "foo", "bar", clientv3.WithLease(clientv3.LeaseID(resp.ID)))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create key with lease %v", err)
|
||||
}
|
||||
@ -64,12 +63,12 @@ func TestLeaseRevoke(t *testing.T) {
|
||||
t.Errorf("failed to create lease %v", err)
|
||||
}
|
||||
|
||||
_, err = lapi.Revoke(context.Background(), lease.LeaseID(resp.ID))
|
||||
_, err = lapi.Revoke(context.Background(), clientv3.LeaseID(resp.ID))
|
||||
if err != nil {
|
||||
t.Errorf("failed to revoke lease %v", err)
|
||||
}
|
||||
|
||||
_, err = kv.Put(context.TODO(), "foo", "bar", clientv3.WithLease(lease.LeaseID(resp.ID)))
|
||||
_, err = kv.Put(context.TODO(), "foo", "bar", clientv3.WithLease(clientv3.LeaseID(resp.ID)))
|
||||
if err != rpctypes.ErrLeaseNotFound {
|
||||
t.Fatalf("err = %v, want %v", err, rpctypes.ErrLeaseNotFound)
|
||||
}
|
||||
@ -89,7 +88,7 @@ func TestLeaseKeepAliveOnce(t *testing.T) {
|
||||
t.Errorf("failed to create lease %v", err)
|
||||
}
|
||||
|
||||
_, err = lapi.KeepAliveOnce(context.Background(), lease.LeaseID(resp.ID))
|
||||
_, err = lapi.KeepAliveOnce(context.Background(), clientv3.LeaseID(resp.ID))
|
||||
if err != nil {
|
||||
t.Errorf("failed to keepalive lease %v", err)
|
||||
}
|
||||
@ -108,7 +107,7 @@ func TestLeaseKeepAlive(t *testing.T) {
|
||||
t.Errorf("failed to create lease %v", err)
|
||||
}
|
||||
|
||||
rc, kerr := lapi.KeepAlive(context.Background(), lease.LeaseID(resp.ID))
|
||||
rc, kerr := lapi.KeepAlive(context.Background(), clientv3.LeaseID(resp.ID))
|
||||
if kerr != nil {
|
||||
t.Errorf("failed to keepalive lease %v", kerr)
|
||||
}
|
||||
@ -148,7 +147,7 @@ func TestLeaseKeepAliveHandleFailure(t *testing.T) {
|
||||
t.Errorf("failed to create lease %v", err)
|
||||
}
|
||||
|
||||
rc, kerr := lapi.KeepAlive(context.Background(), lease.LeaseID(resp.ID))
|
||||
rc, kerr := lapi.KeepAlive(context.Background(), clientv3.LeaseID(resp.ID))
|
||||
if kerr != nil {
|
||||
t.Errorf("failed to keepalive lease %v", kerr)
|
||||
}
|
||||
|
@ -21,18 +21,20 @@ import (
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/google.golang.org/grpc"
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/lease"
|
||||
)
|
||||
|
||||
type (
|
||||
LeaseCreateResponse pb.LeaseCreateResponse
|
||||
LeaseRevokeResponse pb.LeaseRevokeResponse
|
||||
LeaseKeepAliveResponse pb.LeaseKeepAliveResponse
|
||||
LeaseID int64
|
||||
)
|
||||
|
||||
const (
|
||||
// a small buffer to store unsent lease responses.
|
||||
leaseResponseChSize = 16
|
||||
// NoLease is a lease ID for the absence of a lease.
|
||||
NoLease LeaseID = 0
|
||||
)
|
||||
|
||||
type Lease interface {
|
||||
@ -40,14 +42,14 @@ type Lease interface {
|
||||
Create(ctx context.Context, ttl int64) (*LeaseCreateResponse, error)
|
||||
|
||||
// Revoke revokes the given lease.
|
||||
Revoke(ctx context.Context, id lease.LeaseID) (*LeaseRevokeResponse, error)
|
||||
Revoke(ctx context.Context, id LeaseID) (*LeaseRevokeResponse, error)
|
||||
|
||||
// KeepAlive keeps the given lease alive forever.
|
||||
KeepAlive(ctx context.Context, id lease.LeaseID) (<-chan *LeaseKeepAliveResponse, error)
|
||||
KeepAlive(ctx context.Context, id LeaseID) (<-chan *LeaseKeepAliveResponse, error)
|
||||
|
||||
// KeepAliveOnce renews the lease once. In most of the cases, Keepalive
|
||||
// should be used instead of KeepAliveOnce.
|
||||
KeepAliveOnce(ctx context.Context, id lease.LeaseID) (*LeaseKeepAliveResponse, error)
|
||||
KeepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAliveResponse, error)
|
||||
|
||||
// Close releases all resources Lease keeps for efficient communication
|
||||
// with the etcd server.
|
||||
@ -71,7 +73,7 @@ type lessor struct {
|
||||
stopCtx context.Context
|
||||
stopCancel context.CancelFunc
|
||||
|
||||
keepAlives map[lease.LeaseID]*keepAlive
|
||||
keepAlives map[LeaseID]*keepAlive
|
||||
}
|
||||
|
||||
// keepAlive multiplexes a keepalive for a lease over multiple channels
|
||||
@ -90,7 +92,7 @@ func NewLease(c *Client) Lease {
|
||||
conn: c.ActiveConnection(),
|
||||
|
||||
donec: make(chan struct{}),
|
||||
keepAlives: make(map[lease.LeaseID]*keepAlive),
|
||||
keepAlives: make(map[LeaseID]*keepAlive),
|
||||
}
|
||||
|
||||
l.remote = pb.NewLeaseClient(l.conn)
|
||||
@ -121,7 +123,7 @@ func (l *lessor) Create(ctx context.Context, ttl int64) (*LeaseCreateResponse, e
|
||||
}
|
||||
}
|
||||
|
||||
func (l *lessor) Revoke(ctx context.Context, id lease.LeaseID) (*LeaseRevokeResponse, error) {
|
||||
func (l *lessor) Revoke(ctx context.Context, id LeaseID) (*LeaseRevokeResponse, error) {
|
||||
cctx, cancel := context.WithCancel(ctx)
|
||||
done := cancelWhenStop(cancel, l.stopCtx.Done())
|
||||
defer close(done)
|
||||
@ -143,7 +145,7 @@ func (l *lessor) Revoke(ctx context.Context, id lease.LeaseID) (*LeaseRevokeResp
|
||||
}
|
||||
}
|
||||
|
||||
func (l *lessor) KeepAlive(ctx context.Context, id lease.LeaseID) (<-chan *LeaseKeepAliveResponse, error) {
|
||||
func (l *lessor) KeepAlive(ctx context.Context, id LeaseID) (<-chan *LeaseKeepAliveResponse, error) {
|
||||
ch := make(chan *LeaseKeepAliveResponse, leaseResponseChSize)
|
||||
|
||||
l.mu.Lock()
|
||||
@ -169,7 +171,7 @@ func (l *lessor) KeepAlive(ctx context.Context, id lease.LeaseID) (<-chan *Lease
|
||||
return ch, nil
|
||||
}
|
||||
|
||||
func (l *lessor) KeepAliveOnce(ctx context.Context, id lease.LeaseID) (*LeaseKeepAliveResponse, error) {
|
||||
func (l *lessor) KeepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAliveResponse, error) {
|
||||
cctx, cancel := context.WithCancel(ctx)
|
||||
done := cancelWhenStop(cancel, l.stopCtx.Done())
|
||||
defer close(done)
|
||||
@ -193,7 +195,7 @@ func (l *lessor) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *lessor) keepAliveCtxCloser(id lease.LeaseID, ctx context.Context, donec <-chan struct{}) {
|
||||
func (l *lessor) keepAliveCtxCloser(id LeaseID, ctx context.Context, donec <-chan struct{}) {
|
||||
select {
|
||||
case <-donec:
|
||||
return
|
||||
@ -225,7 +227,7 @@ func (l *lessor) keepAliveCtxCloser(id lease.LeaseID, ctx context.Context, donec
|
||||
}
|
||||
}
|
||||
|
||||
func (l *lessor) keepAliveOnce(ctx context.Context, id lease.LeaseID) (*LeaseKeepAliveResponse, error) {
|
||||
func (l *lessor) keepAliveOnce(ctx context.Context, id LeaseID) (*LeaseKeepAliveResponse, error) {
|
||||
stream, err := l.getRemote().LeaseKeepAlive(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -251,7 +253,7 @@ func (l *lessor) recvKeepAliveLoop() {
|
||||
for _, ka := range l.keepAlives {
|
||||
ka.Close()
|
||||
}
|
||||
l.keepAlives = make(map[lease.LeaseID]*keepAlive)
|
||||
l.keepAlives = make(map[LeaseID]*keepAlive)
|
||||
l.mu.Unlock()
|
||||
}()
|
||||
|
||||
@ -281,7 +283,7 @@ func (l *lessor) resetRecv() (pb.Lease_LeaseKeepAliveClient, error) {
|
||||
|
||||
// recvKeepAlive updates a lease based on its LeaseKeepAliveResponse
|
||||
func (l *lessor) recvKeepAlive(resp *pb.LeaseKeepAliveResponse) {
|
||||
id := lease.LeaseID(resp.ID)
|
||||
id := LeaseID(resp.ID)
|
||||
|
||||
l.mu.Lock()
|
||||
defer l.mu.Unlock()
|
||||
@ -320,7 +322,7 @@ func (l *lessor) sendKeepAliveLoop(stream pb.Lease_LeaseKeepAliveClient) {
|
||||
return
|
||||
}
|
||||
|
||||
tosend := make([]lease.LeaseID, 0)
|
||||
tosend := make([]LeaseID, 0)
|
||||
|
||||
now := time.Now()
|
||||
l.mu.Lock()
|
||||
|
@ -16,7 +16,6 @@ package clientv3
|
||||
|
||||
import (
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/lease"
|
||||
)
|
||||
|
||||
type opType int
|
||||
@ -51,7 +50,7 @@ type Op struct {
|
||||
|
||||
// for put
|
||||
val []byte
|
||||
leaseID lease.LeaseID
|
||||
leaseID LeaseID
|
||||
}
|
||||
|
||||
func (op Op) toRequestUnion() *pb.RequestUnion {
|
||||
@ -146,7 +145,7 @@ func (op *Op) applyOpts(opts []OpOption) {
|
||||
type OpOption func(*Op)
|
||||
|
||||
// WithLease attaches a lease ID to a key in 'Put' request.
|
||||
func WithLease(leaseID lease.LeaseID) OpOption {
|
||||
func WithLease(leaseID LeaseID) OpOption {
|
||||
return func(op *Op) { op.leaseID = leaseID }
|
||||
}
|
||||
|
||||
|
@ -22,7 +22,6 @@ import (
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
|
||||
v3 "github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/clientv3/concurrency"
|
||||
"github.com/coreos/etcd/lease"
|
||||
)
|
||||
|
||||
// Key is a key/revision pair created by the client and stored on etcd
|
||||
@ -33,11 +32,11 @@ type RemoteKV struct {
|
||||
val string
|
||||
}
|
||||
|
||||
func NewKey(kv v3.KV, key string, leaseID lease.LeaseID) (*RemoteKV, error) {
|
||||
func NewKey(kv v3.KV, key string, leaseID v3.LeaseID) (*RemoteKV, error) {
|
||||
return NewKV(kv, key, "", leaseID)
|
||||
}
|
||||
|
||||
func NewKV(kv v3.KV, key, val string, leaseID lease.LeaseID) (*RemoteKV, error) {
|
||||
func NewKV(kv v3.KV, key, val string, leaseID v3.LeaseID) (*RemoteKV, error) {
|
||||
rev, err := putNewKV(kv, key, val, leaseID)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -63,7 +62,7 @@ func NewUniqueKey(kv v3.KV, prefix string) (*RemoteKV, error) {
|
||||
return NewUniqueKV(kv, prefix, "", 0)
|
||||
}
|
||||
|
||||
func NewUniqueKV(kv v3.KV, prefix string, val string, leaseID lease.LeaseID) (*RemoteKV, error) {
|
||||
func NewUniqueKV(kv v3.KV, prefix string, val string, leaseID v3.LeaseID) (*RemoteKV, error) {
|
||||
for {
|
||||
newKey := fmt.Sprintf("%s/%v", prefix, time.Now().UnixNano())
|
||||
rev, err := putNewKV(kv, newKey, val, 0)
|
||||
@ -78,7 +77,7 @@ func NewUniqueKV(kv v3.KV, prefix string, val string, leaseID lease.LeaseID) (*R
|
||||
|
||||
// putNewKV attempts to create the given key, only succeeding if the key did
|
||||
// not yet exist.
|
||||
func putNewKV(kv v3.KV, key, val string, leaseID lease.LeaseID) (int64, error) {
|
||||
func putNewKV(kv v3.KV, key, val string, leaseID v3.LeaseID) (int64, error) {
|
||||
cmp := v3.Compare(v3.Version(key), "=", 0)
|
||||
req := v3.OpPut(key, val, v3.WithLease(leaseID))
|
||||
txnresp, err := kv.Txn(context.TODO()).If(cmp).Then(req).Commit()
|
||||
@ -98,7 +97,7 @@ func NewSequentialKV(kv v3.KV, prefix, val string) (*RemoteKV, error) {
|
||||
|
||||
// newSequentialKV allocates a new sequential key <prefix>/nnnnn with a given
|
||||
// value and lease. Note: a bookkeeping node __<prefix> is also allocated.
|
||||
func newSequentialKV(kv v3.KV, prefix, val string, leaseID lease.LeaseID) (*RemoteKV, error) {
|
||||
func newSequentialKV(kv v3.KV, prefix, val string, leaseID v3.LeaseID) (*RemoteKV, error) {
|
||||
resp, err := kv.Get(context.TODO(), prefix, v3.WithLastKey()...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -21,7 +21,7 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/github.com/spf13/cobra"
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
|
||||
"github.com/coreos/etcd/lease"
|
||||
v3 "github.com/coreos/etcd/clientv3"
|
||||
)
|
||||
|
||||
// NewLeaseCommand returns the cobra command for "lease".
|
||||
@ -92,7 +92,7 @@ func leaseRevokeCommandFunc(cmd *cobra.Command, args []string) {
|
||||
ExitWithError(ExitBadArgs, fmt.Errorf("bad lease ID arg (%v), expecting ID in Hex", err))
|
||||
}
|
||||
|
||||
_, err = mustClientFromCmd(cmd).Revoke(context.TODO(), lease.LeaseID(id))
|
||||
_, err = mustClientFromCmd(cmd).Revoke(context.TODO(), v3.LeaseID(id))
|
||||
if err != nil {
|
||||
fmt.Fprintf(os.Stderr, "failed to revoke lease (%v)\n", err)
|
||||
return
|
||||
@ -123,7 +123,7 @@ func leaseKeepAliveCommandFunc(cmd *cobra.Command, args []string) {
|
||||
ExitWithError(ExitBadArgs, fmt.Errorf("bad lease ID arg (%v), expecting ID in Hex", err))
|
||||
}
|
||||
|
||||
respc, kerr := mustClientFromCmd(cmd).KeepAlive(context.TODO(), lease.LeaseID(id))
|
||||
respc, kerr := mustClientFromCmd(cmd).KeepAlive(context.TODO(), v3.LeaseID(id))
|
||||
if kerr != nil {
|
||||
ExitWithError(ExitBadConnection, kerr)
|
||||
}
|
||||
|
@ -22,7 +22,6 @@ import (
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/github.com/spf13/cobra"
|
||||
"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/lease"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -83,7 +82,7 @@ func getPutOp(cmd *cobra.Command, args []string) (string, string, []clientv3.OpO
|
||||
|
||||
opts := []clientv3.OpOption{}
|
||||
if id != 0 {
|
||||
opts = append(opts, clientv3.WithLease(lease.LeaseID(id)))
|
||||
opts = append(opts, clientv3.WithLease(clientv3.LeaseID(id)))
|
||||
}
|
||||
|
||||
return key, value, opts
|
||||
|
Loading…
x
Reference in New Issue
Block a user