Browse Source

lease: move storage.NoLease to lease package

Anthony Romano 10 years ago
parent
commit
f9af744be3

+ 5 - 0
lease/lessor.go

@@ -26,6 +26,11 @@ import (
 	"github.com/coreos/etcd/storage/backend"
 )
 
+const (
+	// NoLease is a special LeaseID representing the absence of a lease.
+	NoLease = LeaseID(0)
+)
+
 var (
 	minLeaseTerm = 5 * time.Second
 

+ 4 - 3
storage/consistent_watchable_store_test.go

@@ -17,6 +17,7 @@ package storage
 import (
 	"testing"
 
+	"github.com/coreos/etcd/lease"
 	"github.com/coreos/etcd/storage/backend"
 )
 
@@ -33,7 +34,7 @@ func TestConsistentWatchableStoreConsistentIndex(t *testing.T) {
 	tests := []uint64{1, 2, 3, 5, 10}
 	for i, tt := range tests {
 		idx = indexVal(tt)
-		s.Put([]byte("foo"), []byte("bar"), NoLease)
+		s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
 
 		id := s.TxnBegin()
 		g := s.consistentIndex()
@@ -50,10 +51,10 @@ func TestConsistentWatchableStoreSkip(t *testing.T) {
 	s := newConsistentWatchableStore(b, &idx)
 	defer cleanup(s, b, tmpPath)
 
-	s.Put([]byte("foo"), []byte("bar"), NoLease)
+	s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
 
 	// put is skipped
-	rev := s.Put([]byte("foo"), []byte("bar"), NoLease)
+	rev := s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
 	if rev != 0 {
 		t.Errorf("rev = %d, want 0", rev)
 	}

+ 18 - 18
storage/kv_test.go

@@ -193,9 +193,9 @@ func testKVRangeBadRev(t *testing.T, f rangeFunc) {
 	s := NewStore(b)
 	defer cleanup(s, b, tmpPath)
 
-	s.Put([]byte("foo"), []byte("bar"), NoLease)
-	s.Put([]byte("foo1"), []byte("bar1"), NoLease)
-	s.Put([]byte("foo2"), []byte("bar2"), NoLease)
+	s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
+	s.Put([]byte("foo1"), []byte("bar1"), lease.NoLease)
+	s.Put([]byte("foo2"), []byte("bar2"), lease.NoLease)
 	if err := s.Compact(3); err != nil {
 		t.Fatalf("compact error (%v)", err)
 	}
@@ -332,9 +332,9 @@ func testKVDeleteRange(t *testing.T, f deleteRangeFunc) {
 		b, tmpPath := backend.NewDefaultTmpBackend()
 		s := NewStore(b)
 
-		s.Put([]byte("foo"), []byte("bar"), NoLease)
-		s.Put([]byte("foo1"), []byte("bar1"), NoLease)
-		s.Put([]byte("foo2"), []byte("bar2"), NoLease)
+		s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
+		s.Put([]byte("foo1"), []byte("bar1"), lease.NoLease)
+		s.Put([]byte("foo2"), []byte("bar2"), lease.NoLease)
 
 		n, rev := f(s, tt.key, tt.end)
 		if n != tt.wN || rev != tt.wrev {
@@ -353,7 +353,7 @@ func testKVDeleteMultipleTimes(t *testing.T, f deleteRangeFunc) {
 	s := NewStore(b)
 	defer cleanup(s, b, tmpPath)
 
-	s.Put([]byte("foo"), []byte("bar"), NoLease)
+	s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
 
 	n, rev := f(s, []byte("foo"), nil)
 	if n != 1 || rev != 2 {
@@ -378,7 +378,7 @@ func TestKVOperationInSequence(t *testing.T) {
 		base := int64(i * 2)
 
 		// put foo
-		rev := s.Put([]byte("foo"), []byte("bar"), NoLease)
+		rev := s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
 		if rev != base+1 {
 			t.Errorf("#%d: put rev = %d, want %d", i, rev, base+1)
 		}
@@ -388,7 +388,7 @@ func TestKVOperationInSequence(t *testing.T) {
 			t.Fatal(err)
 		}
 		wkvs := []storagepb.KeyValue{
-			{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: base + 1, ModRevision: base + 1, Version: 1, Lease: int64(NoLease)},
+			{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: base + 1, ModRevision: base + 1, Version: 1, Lease: int64(lease.NoLease)},
 		}
 		if !reflect.DeepEqual(kvs, wkvs) {
 			t.Errorf("#%d: kvs = %+v, want %+v", i, kvs, wkvs)
@@ -423,7 +423,7 @@ func TestKVTxnBlockNonTnxOperations(t *testing.T) {
 
 	tests := []func(){
 		func() { s.Range([]byte("foo"), nil, 0, 0) },
-		func() { s.Put([]byte("foo"), nil, NoLease) },
+		func() { s.Put([]byte("foo"), nil, lease.NoLease) },
 		func() { s.DeleteRange([]byte("foo"), nil) },
 	}
 	for i, tt := range tests {
@@ -462,7 +462,7 @@ func TestKVTxnWrongID(t *testing.T) {
 			return err
 		},
 		func() error {
-			_, err := s.TxnPut(wrongid, []byte("foo"), nil, NoLease)
+			_, err := s.TxnPut(wrongid, []byte("foo"), nil, lease.NoLease)
 			return err
 		},
 		func() error {
@@ -495,7 +495,7 @@ func TestKVTnxOperationInSequence(t *testing.T) {
 		base := int64(i)
 
 		// put foo
-		rev, err := s.TxnPut(id, []byte("foo"), []byte("bar"), NoLease)
+		rev, err := s.TxnPut(id, []byte("foo"), []byte("bar"), lease.NoLease)
 		if err != nil {
 			t.Fatal(err)
 		}
@@ -508,7 +508,7 @@ func TestKVTnxOperationInSequence(t *testing.T) {
 			t.Fatal(err)
 		}
 		wkvs := []storagepb.KeyValue{
-			{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: base + 1, ModRevision: base + 1, Version: 1, Lease: int64(NoLease)},
+			{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: base + 1, ModRevision: base + 1, Version: 1, Lease: int64(lease.NoLease)},
 		}
 		if !reflect.DeepEqual(kvs, wkvs) {
 			t.Errorf("#%d: kvs = %+v, want %+v", i, kvs, wkvs)
@@ -600,9 +600,9 @@ func TestKVCompactBad(t *testing.T) {
 	s := NewStore(b)
 	defer cleanup(s, b, tmpPath)
 
-	s.Put([]byte("foo"), []byte("bar0"), NoLease)
-	s.Put([]byte("foo"), []byte("bar1"), NoLease)
-	s.Put([]byte("foo"), []byte("bar2"), NoLease)
+	s.Put([]byte("foo"), []byte("bar0"), lease.NoLease)
+	s.Put([]byte("foo"), []byte("bar1"), lease.NoLease)
+	s.Put([]byte("foo"), []byte("bar2"), lease.NoLease)
 
 	// rev in tests will be called in Compact() one by one on the same store
 	tests := []struct {
@@ -631,8 +631,8 @@ func TestKVHash(t *testing.T) {
 		var err error
 		b, tmpPath := backend.NewDefaultTmpBackend()
 		kv := NewStore(b)
-		kv.Put([]byte("foo0"), []byte("bar0"), NoLease)
-		kv.Put([]byte("foo1"), []byte("bar0"), NoLease)
+		kv.Put([]byte("foo0"), []byte("bar0"), lease.NoLease)
+		kv.Put([]byte("foo1"), []byte("bar0"), lease.NoLease)
 		hashes[i], err = kv.Hash()
 		if err != nil {
 			t.Fatalf("failed to get hash: %v", err)

+ 0 - 2
storage/kvstore.go

@@ -38,8 +38,6 @@ var (
 	markBytePosition       = markedRevBytesLen - 1
 	markTombstone     byte = 't'
 
-	NoLease = lease.LeaseID(0)
-
 	scheduledCompactKeyName = []byte("scheduledCompactRev")
 	finishedCompactKeyName  = []byte("finishedCompactRev")
 

+ 3 - 2
storage/kvstore_bench_test.go

@@ -17,6 +17,7 @@ import (
 	"log"
 	"testing"
 
+	"github.com/coreos/etcd/lease"
 	"github.com/coreos/etcd/storage/backend"
 )
 
@@ -32,7 +33,7 @@ func BenchmarkStorePut(b *testing.B) {
 
 	b.ResetTimer()
 	for i := 0; i < b.N; i++ {
-		s.Put(keys[i], vals[i], NoLease)
+		s.Put(keys[i], vals[i], lease.NoLease)
 	}
 }
 
@@ -52,7 +53,7 @@ func BenchmarkStoreTxnPut(b *testing.B) {
 	b.ResetTimer()
 	for i := 0; i < b.N; i++ {
 		id := s.TxnBegin()
-		if _, err := s.TxnPut(id, keys[i], vals[i], NoLease); err != nil {
+		if _, err := s.TxnPut(id, keys[i], vals[i], lease.NoLease); err != nil {
 			log.Fatalf("txn put error: %v", err)
 		}
 		s.TxnEnd(id)

+ 5 - 5
storage/kvstore_test.go

@@ -35,7 +35,7 @@ func TestStoreRev(t *testing.T) {
 	defer os.Remove(tmpPath)
 
 	for i := 0; i < 3; i++ {
-		s.Put([]byte("foo"), []byte("bar"), NoLease)
+		s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
 		if r := s.Rev(); r != int64(i+1) {
 			t.Errorf("#%d: rev = %d, want %d", i, r, i+1)
 		}
@@ -363,9 +363,9 @@ func TestRestoreContinueUnfinishedCompaction(t *testing.T) {
 	s0 := NewStore(b)
 	defer os.Remove(tmpPath)
 
-	s0.Put([]byte("foo"), []byte("bar"), NoLease)
-	s0.Put([]byte("foo"), []byte("bar1"), NoLease)
-	s0.Put([]byte("foo"), []byte("bar2"), NoLease)
+	s0.Put([]byte("foo"), []byte("bar"), lease.NoLease)
+	s0.Put([]byte("foo"), []byte("bar1"), lease.NoLease)
+	s0.Put([]byte("foo"), []byte("bar2"), lease.NoLease)
 
 	// write scheduled compaction, but not do compaction
 	rbytes := newRevBytes()
@@ -423,7 +423,7 @@ func TestTxnPut(t *testing.T) {
 		id := s.TxnBegin()
 		base := int64(i + 1)
 
-		rev, err := s.TxnPut(id, keys[i], vals[i], NoLease)
+		rev, err := s.TxnPut(id, keys[i], vals[i], lease.NoLease)
 		if err != nil {
 			t.Error("txn put error")
 		}

+ 3 - 2
storage/watchable_store_bench_test.go

@@ -19,6 +19,7 @@ import (
 	"os"
 	"testing"
 
+	"github.com/coreos/etcd/lease"
 	"github.com/coreos/etcd/storage/backend"
 )
 
@@ -57,7 +58,7 @@ func BenchmarkWatchableStoreUnsyncedCancel(b *testing.B) {
 	// and force watchers to be in unsynced.
 	testKey := []byte("foo")
 	testValue := []byte("bar")
-	s.Put(testKey, testValue, NoLease)
+	s.Put(testKey, testValue, lease.NoLease)
 
 	w := ws.NewWatchStream()
 
@@ -98,7 +99,7 @@ func BenchmarkWatchableStoreSyncedCancel(b *testing.B) {
 	// Put a key so that we can spawn watchers on that key
 	testKey := []byte("foo")
 	testValue := []byte("bar")
-	s.Put(testKey, testValue, NoLease)
+	s.Put(testKey, testValue, lease.NoLease)
 
 	w := s.NewWatchStream()
 

+ 6 - 5
storage/watchable_store_test.go

@@ -20,6 +20,7 @@ import (
 	"reflect"
 	"testing"
 
+	"github.com/coreos/etcd/lease"
 	"github.com/coreos/etcd/storage/backend"
 	"github.com/coreos/etcd/storage/storagepb"
 )
@@ -35,7 +36,7 @@ func TestWatch(t *testing.T) {
 
 	testKey := []byte("foo")
 	testValue := []byte("bar")
-	s.Put(testKey, testValue, NoLease)
+	s.Put(testKey, testValue, lease.NoLease)
 
 	w := s.NewWatchStream()
 	w.Watch(testKey, true, 0)
@@ -56,7 +57,7 @@ func TestNewWatcherCancel(t *testing.T) {
 	}()
 	testKey := []byte("foo")
 	testValue := []byte("bar")
-	s.Put(testKey, testValue, NoLease)
+	s.Put(testKey, testValue, lease.NoLease)
 
 	w := s.NewWatchStream()
 	wt := w.Watch(testKey, true, 0)
@@ -99,7 +100,7 @@ func TestCancelUnsynced(t *testing.T) {
 	// and force watchers to be in unsynced.
 	testKey := []byte("foo")
 	testValue := []byte("bar")
-	s.Put(testKey, testValue, NoLease)
+	s.Put(testKey, testValue, lease.NoLease)
 
 	w := s.NewWatchStream()
 
@@ -147,7 +148,7 @@ func TestSyncWatchers(t *testing.T) {
 
 	testKey := []byte("foo")
 	testValue := []byte("bar")
-	s.Put(testKey, testValue, NoLease)
+	s.Put(testKey, testValue, lease.NoLease)
 
 	w := s.NewWatchStream()
 
@@ -223,7 +224,7 @@ func TestUnsafeAddWatcher(t *testing.T) {
 	}()
 	testKey := []byte("foo")
 	testValue := []byte("bar")
-	s.Put(testKey, testValue, NoLease)
+	s.Put(testKey, testValue, lease.NoLease)
 
 	size := 10
 	ws := make([]*watcher, size)

+ 3 - 2
storage/watcher_test.go

@@ -17,6 +17,7 @@ package storage
 import (
 	"testing"
 
+	"github.com/coreos/etcd/lease"
 	"github.com/coreos/etcd/storage/backend"
 )
 
@@ -39,7 +40,7 @@ func TestWatcherWatchID(t *testing.T) {
 		}
 		idm[id] = struct{}{}
 
-		s.Put([]byte("foo"), []byte("bar"), NoLease)
+		s.Put([]byte("foo"), []byte("bar"), lease.NoLease)
 
 		resp := <-w.Chan()
 		if resp.WatchID != id {
@@ -51,7 +52,7 @@ func TestWatcherWatchID(t *testing.T) {
 		}
 	}
 
-	s.Put([]byte("foo2"), []byte("bar"), NoLease)
+	s.Put([]byte("foo2"), []byte("bar"), lease.NoLease)
 
 	// unsynced watchers
 	for i := 10; i < 20; i++ {

+ 3 - 3
tools/benchmark/cmd/storage-put.go

@@ -21,7 +21,7 @@ import (
 	"time"
 
 	"github.com/coreos/etcd/Godeps/_workspace/src/github.com/spf13/cobra"
-	"github.com/coreos/etcd/storage"
+	"github.com/coreos/etcd/lease"
 )
 
 // storagePutCmd represents a storage put performance benchmarking tool
@@ -73,13 +73,13 @@ func storagePutFunc(cmd *cobra.Command, args []string) {
 
 		if txn {
 			id := s.TxnBegin()
-			if _, err := s.TxnPut(id, keys[i], vals[i], storage.NoLease); err != nil {
+			if _, err := s.TxnPut(id, keys[i], vals[i], lease.NoLease); err != nil {
 				fmt.Errorf("txn put error: %v", err)
 				os.Exit(1)
 			}
 			s.TxnEnd(id)
 		} else {
-			s.Put(keys[i], vals[i], storage.NoLease)
+			s.Put(keys[i], vals[i], lease.NoLease)
 		}
 
 		end := time.Now()