소스 검색

contrib/raftexample: rename internal packages

Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
Gyuho Lee 7 년 전
부모
커밋
04df89bc11
2개의 변경된 파일11개의 추가작업 그리고 11개의 파일을 삭제
  1. 4 4
      contrib/raftexample/kvstore.go
  2. 7 7
      contrib/raftexample/raft.go

+ 4 - 4
contrib/raftexample/kvstore.go

@@ -21,7 +21,7 @@ import (
 	"log"
 	"sync"
 
-	"github.com/coreos/etcd/raftsnap"
+	"github.com/coreos/etcd/etcdserver/api/snap"
 )
 
 // a key-value store backed by raft
@@ -29,7 +29,7 @@ type kvstore struct {
 	proposeC    chan<- string // channel for proposing updates
 	mu          sync.RWMutex
 	kvStore     map[string]string // current committed key-value pairs
-	snapshotter *raftsnap.Snapshotter
+	snapshotter *snap.Snapshotter
 }
 
 type kv struct {
@@ -37,7 +37,7 @@ type kv struct {
 	Val string
 }
 
-func newKVStore(snapshotter *raftsnap.Snapshotter, proposeC chan<- string, commitC <-chan *string, errorC <-chan error) *kvstore {
+func newKVStore(snapshotter *snap.Snapshotter, proposeC chan<- string, commitC <-chan *string, errorC <-chan error) *kvstore {
 	s := &kvstore{proposeC: proposeC, kvStore: make(map[string]string), snapshotter: snapshotter}
 	// replay log into key-value map
 	s.readCommits(commitC, errorC)
@@ -67,7 +67,7 @@ func (s *kvstore) readCommits(commitC <-chan *string, errorC <-chan error) {
 			// done replaying log; new data incoming
 			// OR signaled to load snapshot
 			snapshot, err := s.snapshotter.Load()
-			if err == raftsnap.ErrNoSnapshot {
+			if err == snap.ErrNoSnapshot {
 				return
 			}
 			if err != nil {

+ 7 - 7
contrib/raftexample/raft.go

@@ -24,13 +24,13 @@ import (
 	"strconv"
 	"time"
 
+	"github.com/coreos/etcd/etcdserver/api/snap"
 	stats "github.com/coreos/etcd/etcdserver/api/v2stats"
 	"github.com/coreos/etcd/pkg/fileutil"
 	"github.com/coreos/etcd/pkg/types"
 	"github.com/coreos/etcd/raft"
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/coreos/etcd/rafthttp"
-	"github.com/coreos/etcd/raftsnap"
 	"github.com/coreos/etcd/wal"
 	"github.com/coreos/etcd/wal/walpb"
 
@@ -61,8 +61,8 @@ type raftNode struct {
 	raftStorage *raft.MemoryStorage
 	wal         *wal.WAL
 
-	snapshotter      *raftsnap.Snapshotter
-	snapshotterReady chan *raftsnap.Snapshotter // signals when snapshotter is ready
+	snapshotter      *snap.Snapshotter
+	snapshotterReady chan *snap.Snapshotter // signals when snapshotter is ready
 
 	snapCount uint64
 	transport *rafthttp.Transport
@@ -79,7 +79,7 @@ var defaultSnapshotCount uint64 = 10000
 // commit channel, followed by a nil message (to indicate the channel is
 // current), then new log entries. To shutdown, close proposeC and read errorC.
 func newRaftNode(id int, peers []string, join bool, getSnapshot func() ([]byte, error), proposeC <-chan string,
-	confChangeC <-chan raftpb.ConfChange) (<-chan *string, <-chan error, <-chan *raftsnap.Snapshotter) {
+	confChangeC <-chan raftpb.ConfChange) (<-chan *string, <-chan error, <-chan *snap.Snapshotter) {
 
 	commitC := make(chan *string)
 	errorC := make(chan error)
@@ -100,7 +100,7 @@ func newRaftNode(id int, peers []string, join bool, getSnapshot func() ([]byte,
 		httpstopc:   make(chan struct{}),
 		httpdonec:   make(chan struct{}),
 
-		snapshotterReady: make(chan *raftsnap.Snapshotter, 1),
+		snapshotterReady: make(chan *snap.Snapshotter, 1),
 		// rest of structure populated after WAL replay
 	}
 	go rc.startRaft()
@@ -190,7 +190,7 @@ func (rc *raftNode) publishEntries(ents []raftpb.Entry) bool {
 
 func (rc *raftNode) loadSnapshot() *raftpb.Snapshot {
 	snapshot, err := rc.snapshotter.Load()
-	if err != nil && err != raftsnap.ErrNoSnapshot {
+	if err != nil && err != snap.ErrNoSnapshot {
 		log.Fatalf("raftexample: error loading snapshot (%v)", err)
 	}
 	return snapshot
@@ -263,7 +263,7 @@ func (rc *raftNode) startRaft() {
 			log.Fatalf("raftexample: cannot create dir for snapshot (%v)", err)
 		}
 	}
-	rc.snapshotter = raftsnap.New(zap.NewExample(), rc.snapdir)
+	rc.snapshotter = snap.New(zap.NewExample(), rc.snapdir)
 	rc.snapshotterReady <- rc.snapshotter
 
 	oldwal := wal.Exist(rc.waldir)