diff --git a/.travis.yml b/.travis.yml
index a2fb463..8c71309 100755
--- a/.travis.yml
+++ b/.travis.yml
@@ -9,6 +9,7 @@ jobs:
       env: GO111MODULE=on TEST_PLATFORM=binary TEST_GENESIS_IMAGE=$GOPATH/src/github.com/dappledger/AnnChain/build/genesis TEST_CONSENSUS_TYPE=pbft
       before_script:
         - make
+        - make image
       script:
         - make test
         - go test ./gemmill/... -coverprofile=coverage.txt -covermode=atomic
diff --git a/Dockerfile b/Dockerfile
new file mode 100644
index 0000000..3c94239
--- /dev/null
+++ b/Dockerfile
@@ -0,0 +1,13 @@
+# compile environment;
+FROM annchain/builder:go1.12 as builder
+#copy files;
+ADD . /AnnChain
+WORKDIR /AnnChain
+RUN GO111MODULE="on" GOPROXY="https://goproxy.io" make genesis
+
+# package environment;
+FROM annchain/runner:alpine3.11
+WORKDIR /genesis
+COPY --from=builder /AnnChain/build/genesis /bin/
+ENTRYPOINT ["genesis"]
+
diff --git a/Makefile b/Makefile
index 5cc7b8f..4624044 100644
--- a/Makefile
+++ b/Makefile
@@ -49,6 +49,7 @@ test-gemmill:
 	go test -v ./gemmill/modules/go-common
 	go test -v ./gemmill/modules/go-db
 	go test -v ./gemmill/modules/go-events
+	# 'go test flowrate' failed maybe your machine upgrade required.
 	go test -v ./gemmill/modules/go-flowrate/flowrate
 	go test -v ./gemmill/modules/go-log
 	go test -v ./gemmill/modules/go-merkle
@@ -56,3 +57,15 @@ test-gemmill:
 	go test -v ./gemmill/refuse_list
 	go test -v ./gemmill/types
 	go test -v ./gemmill/utils
+
+image:
+	docker build -t genesis:latest -f Dockerfile .
+	docker tag genesis:latest annchain/genesis:latest
+
+# docker build and run
+fastrun:image
+	docker-compose -f docker-compose.yaml up
+
+clean_fastrun:
+	docker-compose -f docker-compose.yaml stop
+	docker-compose -f docker-compose.yaml rm
\ No newline at end of file
diff --git a/README.md b/README.md
index ebba75a..2c2471b 100644
--- a/README.md
+++ b/README.md
@@ -122,6 +122,8 @@ consensus = "raft"
 #### Single node
 
 ``` shell
+./build.sh genesis
+
 ./build/genesis init
 
 ./build/genesis run
@@ -130,11 +132,11 @@ consensus = "raft"
 #### Local cluster using docker-compose
 
 ``` shell
-## start cluster
-➜  docker-compose up
+# docker build image and docker-compose run
+make fastrun
 
-## remove cluster
-➜  docker-compose down
+# remove cluster
+make clean_fastrun
 ```
 
 ## Usage
diff --git a/README_CN.md b/README_CN.md
index 50716f1..1e58387 100644
--- a/README_CN.md
+++ b/README_CN.md
@@ -121,6 +121,8 @@ consensus = "raft"
 #### 单节点
 
 ``` shell
+./build.sh genesis
+
 ./build/genesis init
 
 ./build/genesis run
@@ -129,11 +131,11 @@ consensus = "raft"
 #### 使用docker-compose的本地集群
 
 ``` shell
-## start cluster
-➜  docker-compose up
+# docker build image and docker-compose run
+make fastrun
 
-## remove cluster
-➜  docker-compose down
+# remove cluster
+make clean_fastrun
 ```
 
 ## 用法
diff --git a/chain/app/evm/evm.go b/chain/app/evm/evm.go
index f7435d6..73f936e 100755
--- a/chain/app/evm/evm.go
+++ b/chain/app/evm/evm.go
@@ -104,14 +104,16 @@ type EVMApp struct {
 	currentHeader *etypes.Header
 	chainConfig   *params.ChainConfig
 
-	stateDb      ethdb.Database
-	stateMtx     sync.Mutex
-	state        *estate.StateDB
-	currentState *estate.StateDB
+	stateDb                ethdb.Database
+	keyValueHistoryManager *KeyValueHistoryManager
+	stateMtx               sync.Mutex
+	state                  *estate.StateDB
+	currentState           *estate.StateDB
 
-	receipts etypes.Receipts
-	kvs      rtypes.KVs
-	Signer   etypes.Signer
+	receipts          etypes.Receipts
+	kvs               rtypes.KVs
+	keyValueHistories gtypes.KeyValueHistories
+	Signer            etypes.Signer
 }
 
 type LastBlockInfo struct {
@@ -144,6 +146,12 @@ func NewEVMApp(config *viper.Viper) (*EVMApp, error) {
 		log.Error("OpenDatabase error", zap.Error(err))
 		return nil, errors.Wrap(err, "app error")
 	}
+	if kvdb, err := OpenDatabase(app.datadir, "kv_update_history", DatabaseCache, DatabaseHandles); err != nil {
+		log.Error("OpenDatabase error", zap.Error(err))
+		return nil, errors.Wrap(err, "app error")
+	} else {
+		app.keyValueHistoryManager = NewKeyValueHistoryManager(kvdb)
+	}
 
 	app.pool = NewEthTxPool(app, config)
 
@@ -220,6 +228,7 @@ func (app *EVMApp) GetTxPool() gtypes.TxPool {
 func (app *EVMApp) Stop() {
 	app.BaseApplication.Stop()
 	app.stateDb.Close()
+	app.keyValueHistoryManager.Close()
 }
 
 func (app *EVMApp) GetAngineHooks() gtypes.Hooks {
@@ -286,6 +295,7 @@ func (app *EVMApp) genExecFun(block *gtypes.Block, res *gtypes.ExecuteResult) Be
 		stateSnapshot := state.Snapshot()
 		temReceipt := make([]*etypes.Receipt, 0)
 		temKv := make([]*rtypes.KV, 0)
+		tempKeyValueUpdateHistories := make([]*gtypes.KeyValueHistory, 0)
 
 		execFunc := func(txIndex int, raw []byte, tx *etypes.Transaction) error {
 			txType := common.Bytes2Hex(tx.Data())
@@ -295,6 +305,18 @@ func (app *EVMApp) genExecFun(block *gtypes.Block, res *gtypes.ExecuteResult) Be
 					return err
 				}
 				temKv = append(temKv, kv)
+				txBytes, _ := rlp.EncodeToBytes(tx)
+				history := &gtypes.KeyValueHistory{
+					Key: kv.Key,
+					ValueUpdateHistory: &gtypes.ValueUpdateHistory{
+						Value:       kv.Value,
+						TxHash:      gtypes.Tx(txBytes).Hash(),
+						BlockHeight: uint64(block.Height),
+						TimeStamp:   uint64(block.Time.Unix()),
+						TxIndex:uint32(txIndex),
+					},
+				}
+				tempKeyValueUpdateHistories = append(tempKeyValueUpdateHistories, history)
 			} else {
 				receipt, err := app.executeOriginTx(blockHash, state, txIndex, raw, tx)
 				if err != nil {
@@ -317,6 +339,7 @@ func (app *EVMApp) genExecFun(block *gtypes.Block, res *gtypes.ExecuteResult) Be
 			}
 			app.receipts = append(app.receipts, temReceipt...)
 			app.kvs = append(app.kvs, temKv...)
+			app.keyValueHistories = append(app.keyValueHistories, tempKeyValueUpdateHistories...)
 			res.ValidTxs = append(res.ValidTxs, raw)
 			return true
 		}
@@ -398,7 +421,7 @@ func (app *EVMApp) GetAddressFromTx(tx *etypes.Transaction) (from common.Address
 	return
 }
 
-func (app *EVMApp) CheckTx(bs []byte) (from common.Address,nonce uint64, err error) {
+func (app *EVMApp) CheckTx(bs []byte) (from common.Address, nonce uint64, err error) {
 	tx := &etypes.Transaction{}
 	err = rlp.DecodeBytes(bs, &tx)
 	if err != nil {
@@ -436,10 +459,6 @@ func (app *EVMApp) CheckTx(bs []byte) (from common.Address,nonce uint64, err err
 			err = fmt.Errorf("key or value too big,MaxKey:%v,MaxValue:%v", MaxKey, MaxValue)
 			return
 		}
-		if ok, _ := app.stateDb.Has(append(KvPrefix, kvData.Key...)); ok {
-			err = fmt.Errorf("duplicate key :%v", kvData.Key)
-			return
-		}
 	}
 	return
 }
@@ -477,6 +496,12 @@ func (app *EVMApp) SaveReceipts() ([]byte, error) {
 	if err := receiptBatch.Write(); err != nil {
 		return nil, fmt.Errorf("persist receipts failed:%v", err.Error())
 	}
+	err := app.keyValueHistoryManager.SaveKeyHistory(app.keyValueHistories)
+	if err != nil {
+		log.Warnf("save key value history error", zap.Error(err))
+	}
+
+	app.keyValueHistories = nil
 	rHash := merkle.SimpleHashFromHashes(savedReceipts)
 	return rHash, nil
 }
@@ -525,6 +550,13 @@ func (app *EVMApp) Query(query []byte) (res gtypes.Result) {
 		res = app.queryKeyWithPrefix(load)
 	case rtypes.QueryType_Pending_Nonce:
 		res = app.queryPendingNonce(load)
+	case rtypes.QueryType_Key_Update_History:
+		if len(load) < (PageNumLen + PageSizeLen+1) {
+			return gtypes.NewError(gtypes.CodeType_BaseInvalidInput, "wrong pageNo and pageSize")
+		}
+		pageNo := binary.BigEndian.Uint32(load[:PageNumLen])
+		pageSize := binary.BigEndian.Uint32(load[PageNumLen:PageNumLen+PageSizeLen])
+		res = app.queryKeyUpdateHistory(load[PageNumLen+PageSizeLen:], pageNo, pageSize)
 	default:
 		res = gtypes.NewError(gtypes.CodeType_BaseInvalidInput, "unimplemented query")
 	}
@@ -733,6 +765,24 @@ func (app *EVMApp) queryKeyWithPrefix(load []byte) gtypes.Result {
 	return gtypes.NewResultOK(bytKvs, "")
 }
 
+func (app *EVMApp) queryKeyUpdateHistory(key []byte, pageNo uint32, pageSize uint32) gtypes.Result {
+
+	kvs, total, err := app.keyValueHistoryManager.Query(key, pageNo, pageSize)
+	if err != nil {
+		return gtypes.NewError(gtypes.CodeType_InternalError, fmt.Sprintf("fail to get value history  %s %v:",string(key),err))
+	}
+	result := gtypes.ValueHistoryResult{
+		Key:                  key,
+		ValueUpdateHistories: kvs,
+		Total:                total,
+	}
+	byteKvs, err := rlp.EncodeToBytes(result)
+	if err != nil {
+		return gtypes.NewError(gtypes.CodeType_WrongRLP, "rlp encode error:"+err.Error())
+	}
+	return gtypes.NewResultOK(byteKvs, "")
+}
+
 func (app *EVMApp) SetCore(core gtypes.Core) {
 	app.core = core
 }
diff --git a/chain/app/evm/kv.go b/chain/app/evm/kv.go
new file mode 100644
index 0000000..45f5c26
--- /dev/null
+++ b/chain/app/evm/kv.go
@@ -0,0 +1,243 @@
+package evm
+
+import (
+	"bytes"
+	"encoding/binary"
+	"fmt"
+	"sync"
+
+	"github.com/dappledger/AnnChain/eth/ethdb"
+	"github.com/dappledger/AnnChain/eth/rlp"
+	"github.com/dappledger/AnnChain/gemmill/modules/go-log"
+	"github.com/dappledger/AnnChain/gemmill/types"
+	"go.uber.org/zap"
+)
+
+var (
+	KvHistoryPrefix = []byte("kvh_")
+	KvSizePrefix    = []byte("_size_")
+	kvIndexPrefix = []byte("_ff_")
+)
+
+const (
+	PageNumLen = 4
+	PageSizeLen = 4
+)
+
+func makeKey(key []byte, suffix[] byte) []byte {
+	buf:= bytes.NewBuffer(nil)
+	buf.Write(KvHistoryPrefix)
+	buf.Write(key)
+	buf.Write([]byte("_ff_"))
+	buf.Write(suffix)
+	return buf.Bytes()
+}
+
+func makeKeySizeKey(key []byte) []byte {
+	buf:= bytes.NewBuffer(nil)
+	buf.Write(KvHistoryPrefix)
+	buf.Write(key)
+	buf.Write([]byte("_fd_"))
+	buf.Write( KvSizePrefix)
+	return buf.Bytes()
+}
+
+func putUint32(i uint32) []byte {
+	index := make([]byte, 4)
+	binary.BigEndian.PutUint32(index, i)
+	return index
+}
+
+
+type KeyValueHistoryManager struct {
+	db ethdb.Database
+	mu sync.RWMutex
+}
+
+func (m *KeyValueHistoryManager) Close() {
+	m.db.Close()
+}
+
+func NewKeyValueHistoryManager(db ethdb.Database) *KeyValueHistoryManager {
+	return &KeyValueHistoryManager{db: db}
+}
+
+
+func (m *KeyValueHistoryManager) SaveKeyHistory(kvs types.KeyValueHistories) error {
+	if len(kvs) ==0 {
+		return nil
+	}
+	batch := m.NewBatch()
+	return batch.SaveKeyHistory(kvs)
+}
+
+
+func (m *KeyValueHistoryManager) NewBatch() *kvBatch {
+	k := &kvBatch{
+		batch:                  m.db.NewBatch(),
+		keys:                   make(map[string]uint32),
+		KeyValueHistoryManager: m,
+	}
+	return k
+}
+
+func (m *KeyValueHistoryManager) GetKeyHistorySize(key []byte) (uint32, error) {
+	m.mu.RLock()
+	defer m.mu.RUnlock()
+	return m.getKeyHistorySize(key)
+}
+
+func (m *KeyValueHistoryManager) getKeyHistorySize(key []byte) (uint32, error) {
+	k := makeKeySizeKey(key)
+	data, err := m.db.Get(k)
+	if err != nil {
+		return 0, err
+	}
+	if len(data) < 4 {
+		return 0, fmt.Errorf("should be 4 byte %v", data)
+	}
+	return binary.BigEndian.Uint32(data), nil
+}
+
+func (m *KeyValueHistoryManager) Get(key []byte, index uint32) (history *types.ValueUpdateHistory, err error) {
+	m.mu.Lock()
+	defer m.mu.Unlock()
+	return m.get(key, index)
+}
+
+func (m *KeyValueHistoryManager) get(key []byte, index uint32) (*types.ValueUpdateHistory, error) {
+	k := makeKey(key, putUint32(index))
+	data, err := m.db.Get(k)
+	if err != nil {
+		return nil, err
+	}
+	history := &types.ValueUpdateHistory{}
+	err = rlp.DecodeBytes(data, history)
+	if err != nil {
+		return nil, err
+	}
+	return history, nil
+}
+
+
+func (m *KeyValueHistoryManager) Query(key []byte, pageNo uint32, pageSize uint32) (histories []*types.ValueUpdateHistory, total uint32, err error) {
+	if pageNo < 1 {
+		pageNo = 1
+	}
+	if pageSize == 0 {
+		pageSize = 10
+	}
+	if pageSize >20 {
+		pageSize = 20
+	}
+	total, err = m.GetKeyHistorySize(key)
+	if err != nil {
+		log.Infof("get size err %v", err)
+		return nil, 0, err
+	}
+	if total == 0 {
+		return nil, 0, nil
+	}
+	var from ,to uint32
+	if total < (pageNo-1)*pageSize {
+		return nil, total, nil
+	}
+	from = total - (pageNo-1)*pageSize
+
+	if from >= pageSize {
+		to  = from - pageSize
+	}else {
+		pageSize = from
+	}
+
+	var kvs []*ethdb.KVResult
+	kvs, err = m.db.GetWithPrefix(makeKey(key, nil), makeKey(key, putUint32(to)), pageSize, 0)
+	if err != nil {
+		log.Infof("get key history err for key %v   to %v pageSize %v  err %v",string(key),to,pageSize,err)
+		return nil, total, err
+	}
+	for i := len(kvs) - 1; i >= 0; i-- {
+		kv := kvs[i]
+		history := &types.ValueUpdateHistory{}
+		err = rlp.DecodeBytes(kv.V, &history)
+		if err != nil {
+			log.Infof("decode rlp  err  %v  to %v pageSize %v  err %v %s %s",string(key),to,pageSize,err, i,string(kv.K), string(kv.V))
+			return nil, total, err
+		}
+		histories = append(histories, history)
+	}
+	return
+}
+
+
+type kvBatch struct {
+	batch                  ethdb.Batch
+	keys                   map[string]uint32
+	KeyValueHistoryManager *KeyValueHistoryManager
+}
+
+func (batch *kvBatch) SaveKeyHistory(kvs types.KeyValueHistories) error {
+	batch.KeyValueHistoryManager.mu.Lock()
+	defer batch.KeyValueHistoryManager.mu.Unlock()
+	return batch.saveKeyHistory(kvs)
+}
+
+func (batch *kvBatch) saveKeyHistory(kvs types.KeyValueHistories) error {
+	log.Infof("save key histories %s ",kvs.String())
+	for i := range kvs {
+		kv := kvs[i]
+		err := batch.put(kv.Key, kv.ValueUpdateHistory)
+		if err != nil {
+			return err
+		}
+	}
+	for key, size := range batch.keys {
+		err := batch.setKeyHistorySize([]byte(key), size)
+		if err != nil {
+			log.Warnf("write err %v %v %v", key, size, err)
+			return err
+		}
+	}
+	return batch.batch.Write()
+}
+
+
+func (batch *kvBatch) put(key []byte, history *types.ValueUpdateHistory) error {
+	var size uint32
+	var ok bool
+	var err error
+	if size, ok = batch.keys[string(key)]; !ok {
+		size, err = batch.KeyValueHistoryManager.getKeyHistorySize(key)
+		if err != nil {
+			//check not found
+			log.Info("read size err", zap.Error(err))
+		}
+	}
+	err = batch.putHistory(key, size, history)
+	if err != nil {
+		log.Warnf("write err %v %v %v %v", key, size, err, history)
+		return err
+	}
+	batch.keys[string(key)] = size + 1
+	return nil
+}
+
+func (batch *kvBatch) Put(key []byte, history *types.ValueUpdateHistory) error {
+	batch.KeyValueHistoryManager.mu.Lock()
+	defer batch.KeyValueHistoryManager.mu.Unlock()
+	return batch.put(key, history)
+}
+
+func (batch *kvBatch) setKeyHistorySize(key []byte, size uint32) error {
+	k := makeKeySizeKey(key)
+	return batch.batch.Put(k, putUint32(size))
+}
+
+func (batch *kvBatch) putHistory(key []byte, index uint32, history *types.ValueUpdateHistory) error {
+	k := makeKey(key, putUint32(index))
+	historyBytes, err := rlp.EncodeToBytes(history)
+	if err != nil {
+		return err
+	}
+	return batch.batch.Put(k, historyBytes)
+}
\ No newline at end of file
diff --git a/chain/app/evm/kv_test.go b/chain/app/evm/kv_test.go
new file mode 100644
index 0000000..f8523b3
--- /dev/null
+++ b/chain/app/evm/kv_test.go
@@ -0,0 +1,62 @@
+package evm
+
+import (
+	"fmt"
+	"os"
+	"testing"
+	"time"
+
+	"github.com/dappledger/AnnChain/eth/ethdb"
+	"github.com/dappledger/AnnChain/gemmill/modules/go-log"
+	"github.com/dappledger/AnnChain/gemmill/types"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestLevelDb(t *testing.T) {
+
+	//key 0-1 00
+	//pageNo:10, pageSize:100
+	//iterator  sort
+	//key_size  kry_0 ....key_100
+	//key_90  key_95
+	db, err := ethdb.NewLDBDatabase("_testdb", 0, 0)
+	assert.NoError(t, err)
+	defer func() {
+		time.Sleep(time.Second)
+		os.RemoveAll("_testdb")
+	}()
+	m := NewKeyValueHistoryManager(db)
+	defer m.Close()
+	var histories types.KeyValueHistories
+	genValue := func(val string, suffix string) []byte {
+		return []byte(val + "_" + suffix)
+	}
+	for j := 0; j < 10; j++ {
+		for i := 0; i < 100; i++ {
+			history := &types.KeyValueHistory{
+				Key: []byte(fmt.Sprintf("key_%d", j)),
+				ValueUpdateHistory: &types.ValueUpdateHistory{
+					TimeStamp:   uint64(time.Now().Unix()),
+					Value:       genValue("value", fmt.Sprintf("%d_%d", i, j)),
+					TxHash:      []byte("0x5555"),
+					BlockHeight: 10,
+					TxIndex:12,
+				},
+			}
+			histories = append(histories, history)
+		}
+
+	}
+
+	err = m.SaveKeyHistory(histories)
+	assert.NoError(t, err)
+	result, total, err := m.Query([]byte(fmt.Sprintf("key_%d", 4)), 4, 6)
+	assert.NoError(t, err)
+	assert.Equal(t, 100, int(total))
+	assert.NotNil(t, result)
+	from := 100 - 3*6
+	for i, v := range result {
+		log.Debugf("result %d %v ", i, v, string(v.Value))
+		assert.Equal(t, v.Value, genValue("value", fmt.Sprintf("%d_%d", from-i, 4)))
+	}
+}
diff --git a/chain/commands/init.go b/chain/commands/init.go
index 3b438b4..ebfed74 100644
--- a/chain/commands/init.go
+++ b/chain/commands/init.go
@@ -16,12 +16,10 @@ package commands
 import (
 	"log"
 
-	"github.com/astaxie/beego"
 	"github.com/mitchellh/go-homedir"
 	"github.com/spf13/cobra"
 
 	"github.com/dappledger/AnnChain/chain/commands/global"
-	"github.com/dappledger/AnnChain/chain/commands/vision/routers"
 	"github.com/dappledger/AnnChain/gemmill"
 	"github.com/dappledger/AnnChain/gemmill/go-crypto"
 )
@@ -51,8 +49,6 @@ func NewInitCommand() *cobra.Command {
 
 func newInitCommandFunc(cmd *cobra.Command, args []string) {
 	if visual {
-		routers.InitNode()
-		beego.Run(vport)
 		select {}
 	}
 	if !global.CheckAppName(appName) {
diff --git a/chain/commands/vision/controllers/nodeinit.go b/chain/commands/vision/controllers/nodeinit.go
index 48c4995..462ce7f 100644
--- a/chain/commands/vision/controllers/nodeinit.go
+++ b/chain/commands/vision/controllers/nodeinit.go
@@ -14,14 +14,9 @@
 package controllers
 
 import (
-	"github.com/astaxie/beego"
 	"github.com/dappledger/AnnChain/chain/commands/vision/models"
 )
 
-type InitNode struct {
-	beego.Controller
-}
-
 func (c *InitNode) Get() {
 	c.TplName = "initnode.tpl"
 }
diff --git a/chain/commands/vision/models/nodeinit.go b/chain/commands/vision/models/nodeinit.go
index d9febf7..620066e 100644
--- a/chain/commands/vision/models/nodeinit.go
+++ b/chain/commands/vision/models/nodeinit.go
@@ -19,17 +19,13 @@ import (
 	"errors"
 	"fmt"
 	"os"
-	"time"
 
-	"github.com/astaxie/beego"
 	_ "github.com/mattn/go-sqlite3"
 
 	"github.com/dappledger/AnnChain/chain/commands/global"
-	"github.com/dappledger/AnnChain/chain/core"
 	"github.com/dappledger/AnnChain/gemmill/config"
 	"github.com/dappledger/AnnChain/gemmill/go-crypto"
 	"github.com/dappledger/AnnChain/gemmill/go-utils"
-	"github.com/dappledger/AnnChain/gemmill/modules/go-log"
 	"github.com/dappledger/AnnChain/gemmill/types"
 )
 
@@ -183,44 +179,3 @@ func (n *NodeInit) DoInit() error {
 	}
 	return err
 }
-
-func DoInitNode(c *beego.Controller) {
-	n := &NodeInit{}
-	c.ParseForm(n)
-	if err := n.DoInit(); err != nil {
-		c.Data["json"] = err.Error()
-		return
-	}
-
-	if _, ok := c.Data["json"]; !ok {
-		c.Data["json"] = "Done!"
-	}
-}
-
-func RunNode(c *beego.Controller) {
-	n := &NodeInit{}
-	c.ParseForm(n)
-	err := global.CheckAndReadRuntimeConfig(n.ConfigPath)
-	if err != nil {
-		c.Data["json"] = err.Error()
-		return
-	}
-	chret := make(chan error, 0)
-	go func() {
-		defer log.DumpStack()
-		if err := core.RunNodeRet(global.GConf(), "", global.GConf().GetString("app_name")); err != nil {
-			chret <- err
-		}
-	}()
-	timer := time.NewTimer(time.Second * 2)
-	select {
-	case <-timer.C:
-		c.Data["json"] = "node is running..."
-	case err := <-chret:
-		c.Data["json"] = fmt.Sprintf("Failed to start node: %v", err)
-	}
-}
-
-func CloseServer(c *beego.Controller) {
-	beego.BeeApp.Server.Close()
-}
diff --git a/chain/commands/vision/routers/router.go b/chain/commands/vision/routers/router.go
deleted file mode 100644
index 6b6b13e..0000000
--- a/chain/commands/vision/routers/router.go
+++ /dev/null
@@ -1,23 +0,0 @@
-// Copyright © 2017 ZhongAn Technology
-// 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 routers
-
-import (
-	"github.com/astaxie/beego"
-	"github.com/dappledger/AnnChain/chain/commands/vision/controllers"
-)
-
-func InitNode() {
-	beego.Router("/", &controllers.InitNode{}, "get:Get;post:Post")
-}
diff --git a/chain/types/types.go b/chain/types/types.go
index 0cbbe68..675ab47 100644
--- a/chain/types/types.go
+++ b/chain/types/types.go
@@ -41,19 +41,20 @@ type (
 )
 
 const (
-	APIQueryTx                          = iota
-	QueryType_Contract        QueryType = 0
-	QueryType_Nonce           QueryType = 1
-	QueryType_Balance         QueryType = 2
-	QueryType_Receipt         QueryType = 3
-	QueryType_Existence       QueryType = 4
-	QueryType_PayLoad         QueryType = 5
-	QueryType_TxRaw           QueryType = 6
-	QueryTxLimit              QueryType = 9
-	QueryTypeContractByHeight QueryType = 10
-	QueryType_Key             QueryType = 11
-	QueryType_Key_Prefix      QueryType = 12
-	QueryType_Pending_Nonce   QueryType = 13
+	APIQueryTx                             = iota
+	QueryType_Contract           QueryType = 0
+	QueryType_Nonce              QueryType = 1
+	QueryType_Balance            QueryType = 2
+	QueryType_Receipt            QueryType = 3
+	QueryType_Existence          QueryType = 4
+	QueryType_PayLoad            QueryType = 5
+	QueryType_TxRaw              QueryType = 6
+	QueryTxLimit                 QueryType = 9
+	QueryTypeContractByHeight    QueryType = 10
+	QueryType_Key                QueryType = 11
+	QueryType_Key_Prefix         QueryType = 12
+	QueryType_Pending_Nonce      QueryType = 13
+	QueryType_Key_Update_History QueryType = 14
 )
 
 var KVTxType = []byte("kvTx-")
diff --git a/cmd/client/commands/flags.go b/cmd/client/commands/flags.go
index 1f0eb98..0bdf421 100644
--- a/cmd/client/commands/flags.go
+++ b/cmd/client/commands/flags.go
@@ -29,6 +29,7 @@ type AnntoolFlags struct {
 	hash,
 	accountPubkey,
 	peerPubkey,
+	privateKey,
 	validatorPubkey,
 	validatorSignature,
 	validatorPrivKey,
@@ -38,6 +39,8 @@ type AnntoolFlags struct {
 	cType,
 	verbose,
 	nPrivs,
+	pageNum,
+	pageSize,
 	codeHash cli.Flag
 }
 
@@ -117,4 +120,13 @@ var anntoolFlags = AnntoolFlags{
 		Name:  "nPrivs",
 		Usage: "number of ca privateKey!",
 	},
+	pageNum: cli.UintFlag{
+		Name: "page_num",
+	},
+	pageSize: cli.UintFlag{
+		Name: "page_size",
+	},
+	privateKey: cli.StringFlag{
+		Name: "priv_key",
+	},
 }
diff --git a/cmd/client/commands/kv.go b/cmd/client/commands/kv.go
new file mode 100644
index 0000000..596a54d
--- /dev/null
+++ b/cmd/client/commands/kv.go
@@ -0,0 +1,175 @@
+// Copyright © 2017 ZhongAn Technology
+// 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 commands
+
+import (
+	"encoding/binary"
+	"encoding/json"
+	"fmt"
+	"math/big"
+	"strings"
+
+	rtypes "github.com/dappledger/AnnChain/chain/types"
+	"github.com/dappledger/AnnChain/eth/common"
+	"github.com/dappledger/AnnChain/eth/core/types"
+	"gopkg.in/urfave/cli.v1"
+
+	"github.com/dappledger/AnnChain/cmd/client/commons"
+	"github.com/dappledger/AnnChain/eth/rlp"
+	cl "github.com/dappledger/AnnChain/gemmill/rpc/client"
+	gtypes "github.com/dappledger/AnnChain/gemmill/types"
+)
+
+var (
+	KvPutCommands = cli.Command{
+		Name:     "put",
+		Usage:    "operations for put key value",
+		Category: "put",
+		Action:   putKeyValue,
+		Flags: []cli.Flag{
+			anntoolFlags.privateKey,
+		},
+	}
+	KvGetCommands = cli.Command{
+		Name:     "get",
+		Usage:    "operations for get key value, if",
+		Category: "get",
+		Action:   getKeyValue,
+		Flags: []cli.Flag{
+			anntoolFlags.pageNum,
+			anntoolFlags.pageSize,
+		},
+	}
+)
+
+func getKeyValue(ctx *cli.Context) error {
+	clientJSON := cl.NewClientJSONRPC(commons.QueryServer)
+	rpcResult := new(gtypes.ResultQuery)
+	keyStr := ctx.Args().First()
+	pageNum := ctx.Uint("page_num")
+	if pageNum != 0 {
+		return queryKeyUpdateHistory(ctx)
+	}
+
+	query := append([]byte{rtypes.QueryType_Key}, []byte(keyStr)...)
+
+	_, err := clientJSON.Call("query", []interface{}{query}, rpcResult)
+	if err != nil {
+		return cli.NewExitError(err.Error(), 127)
+	}
+	fmt.Println("query result:", string(rpcResult.Result.Data))
+	return nil
+}
+
+func putKeyValue(ctx *cli.Context) error {
+	clientJSON := cl.NewClientJSONRPC(commons.QueryServer)
+	rpcResult := new(gtypes.ResultBroadcastTxCommit)
+	if ctx.NArg() < 2 {
+		return cli.NewExitError(fmt.Errorf("need key and value"), 127)
+	}
+	keyStr := ctx.Args().First()
+	valueStr := ctx.Args().Get(1)
+	privkey := ctx.String("priv_key")
+	if privkey == "" {
+		return cli.NewExitError("privkey is required", 127)
+	}
+
+	if strings.Index(privkey, "0x") == 0 || strings.Index(privkey, "0X") == 0 {
+		privkey = privkey[2:]
+	}
+
+	privBytes := common.Hex2Bytes(privkey)
+
+	addr, err := getAddrBytes(privBytes)
+	if err != nil {
+		return err
+	}
+
+	nonce, _ := getNonce(addr)
+	kvBytes, err := rlp.EncodeToBytes(&rtypes.KV{Key: []byte(keyStr), Value: []byte(valueStr)})
+	if err != nil {
+		return err
+	}
+
+	txdata := append(rtypes.KVTxType, kvBytes...)
+	tx := types.NewTransaction(nonce, common.Address{}, big.NewInt(0), gasLimit, big.NewInt(0), txdata)
+	signer, sig, err := SignTx(privBytes, tx)
+	if err != nil {
+		return err
+	}
+	sigTx, err := tx.WithSignature(signer, sig)
+	if err != nil {
+		return err
+	}
+
+	b, err := rlp.EncodeToBytes(sigTx)
+	if err != nil {
+		return err
+	}
+
+	_, err = clientJSON.Call("broadcast_tx_commit", []interface{}{b}, rpcResult)
+	if err != nil {
+		return err
+	}
+
+	hash := rpcResult.TxHash
+	fmt.Println("tx result:", hash)
+
+	return nil
+}
+
+func queryKeyUpdateHistory(ctx *cli.Context) error {
+	clientJSON := cl.NewClientJSONRPC(commons.QueryServer)
+	rpcResult := new(gtypes.ResultQuery)
+	keyStr := ctx.Args().First()
+	//ValueHistoryResult
+	pageNum := ctx.Uint("page_num")
+	if pageNum == 0 {
+		pageNum = 1
+	}
+	pageSize := ctx.Uint("page_size")
+	if pageSize == 0 {
+		pageSize = 10
+	}
+	query := append([]byte{rtypes.QueryType_Key_Update_History}, putUint32(uint32(pageNum))...)
+	query = append(query, putUint32(uint32(pageSize))...)
+	query = append(query, []byte(keyStr)...)
+
+	_, err := clientJSON.Call("query", []interface{}{query}, rpcResult)
+	if err != nil {
+		return cli.NewExitError(err.Error(), 127)
+	}
+
+	response := &gtypes.ValueHistoryResult{}
+	err = rlp.DecodeBytes(rpcResult.Result.Data, response)
+	if err != nil {
+		fmt.Println(rpcResult.Result)
+		return cli.NewExitError(err.Error(), 127)
+	}
+
+	responseJSON, err := json.Marshal(response)
+	if err != nil {
+		return cli.NewExitError(err.Error(), 127)
+	}
+
+	fmt.Println("query result:", string(responseJSON))
+
+	return nil
+}
+
+func putUint32(i uint32) []byte {
+	index := make([]byte, 4)
+	binary.BigEndian.PutUint32(index, i)
+	return index
+}
diff --git a/cmd/client/commands/state_query.go b/cmd/client/commands/state_query.go
index dccc19c..91de537 100644
--- a/cmd/client/commands/state_query.go
+++ b/cmd/client/commands/state_query.go
@@ -192,3 +192,4 @@ func getTxByHash(hash []byte) (rt *gtypes.ResultTransaction, ethtx *types.Transa
 
 	return
 }
+
diff --git a/cmd/client/main.go b/cmd/client/main.go
index 83bb786..fbeb6db 100644
--- a/cmd/client/main.go
+++ b/cmd/client/main.go
@@ -41,6 +41,8 @@ func main() {
 		commands.InfoCommand,
 
 		commands.VersionCommands,
+		commands.KvGetCommands,
+		commands.KvPutCommands,
 	}
 
 	app.Flags = []cli.Flag{
diff --git a/docker-compose.yaml b/docker-compose.yaml
index 99621fc..003f209 100644
--- a/docker-compose.yaml
+++ b/docker-compose.yaml
@@ -1,14 +1,5 @@
 version: '3'
 services:
-  mgo:
-    image: mongo:3.2
-    ports:
-      - '27017:27017'
-    command: mongod
-    restart: always
-    networks:
-      app_net:
-        ipv4_address: 192.168.10.7
   validator-0:
     hostname: validator-0
     image: annchain/genesis:latest
diff --git a/docker/build_rendez.sh b/docker/build_rendez.sh
deleted file mode 100755
index f19f1c3..0000000
--- a/docker/build_rendez.sh
+++ /dev/null
@@ -1,14 +0,0 @@
-#!/bin/sh
-
-rm Dockerfile
-echo 'FROM alpine:latest
-RUN apk add --no-cache libc6-compat
-COPY ./genesis /bin/
-
-WORKDIR /genesis
-
-ENTRYPOINT ["genesis"]' > Dockerfile
-
-docker build -t genesis:latest -f Dockerfile ../build
-docker tag genesis:latest annchain/genesis:latest
-rm Dockerfile
\ No newline at end of file
diff --git a/docs/cmd.md b/docs/cmd.md
index 98ce7cf..2f44ae6 100644
--- a/docs/cmd.md
+++ b/docs/cmd.md
@@ -265,6 +265,69 @@ query result receipt
 query result: {"root":null,"status":1,"cumulativeGasUsed":21656,"logsBloom":"0x00000000000000000000000000800000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000200000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002000000000000000000000020000000000000000000000000000000000000000000000000000000000000","logs":[{"address":"0xae119075bd77de2d8e32629bdb439d967a1ecfe6","topics":["0xb45ab3e8c50935ce2fa51d37817fd16e7358a3087fd93a9ac7fbddb22a926c35"],"data":"0x00000000000000000000000000000000000000000000000000000000000000010000000000000000000000000000000000000000000000000000000000000064","blockNumber":"0x64e","transactionHash":"0x2b41d9c05a7be5b85586c53b5a2d3cacc1ded323a18f1c62c51bc2aea0953b55","transactionIndex":"0x0","blockHash":"0x000000000000000000000000ec83a146ca731fdffe4bef69ad260d7389732e87","logIndex":"0x0","removed":false}],"transactionHash":"0x2b41d9c05a7be5b85586c53b5a2d3cacc1ded323a18f1c62c51bc2aea0953b55","contractAddress":"0x0000000000000000000000000000000000000000","gasUsed":21656}
 ```
 
+## KeyValue : get value by key 
+
+####Command 
+```
+ gtool  --backend <validator's IP:RPC Port>  get <key>
+```
+
+##### Result
+
+```
+query result value
+```
+
+##### Demo
+
+```
+./build/gtool --backend "tcp://127.0.0.1:46657" get test_key_0
+query result: v_015
+```
+
+## KeyValue : store a  key value data
+
+
+##### Command
+
+```
+gtool --backend <validator's IP:RPC Port> put <key> <value> --priv_key=<priv_key>
+```
+
+##### Result
+
+```
+tx result 
+```
+
+##### Demo
+
+```
+gtool --backend "tcp://127.0.0.1:46657" put foo bar --priv_key=B37D1C91C2320911EFA1433EC8BBBE48AF15409743F8CB1830708CBBA0DB6690									
+tx result: 0x8eaf236bf0d760c15e798161de84a11c8ed06e7b3622653d85f9dabbd7a44609
+```
+
+## KeyValue  get  key update history 
+
+####Command 
+```
+ gtool  --backend <validator's IP:RPC Port>  get <key> --page_num=<page_num> --page_size=<page_size>
+```
+default page_size is 10, max is 20
+##### Result
+
+```
+query result key_update_history
+```
+
+##### Demo
+
+```
+gtool get foo --page_num=1
+query result: {"key":"Zm9v","value_update_histories":[{"tx_hash":"0sWVAv8hCoMp7qcwsdJ57yOAps2zbFFARAa6H3Ad6ko=","block_height":50283,"time_stamp":1588149308,"value":"YmFyMg==","tx_index":0},{"tx_hash":"3gmcnXK6yQvW4oNbZhV0kEVJtZ/GKIsgP/lcRFmO3aI=","block_height":50273,"time_stamp":1588149294,"value":"YmFyMQ==","tx_index":0}],"total":3}
+```
+
+
 ## New Node Synchronous Chain's Data
 
 If a new node wants to join the chain, it needs to synchronize the chain data in the first. The details are as follows:
diff --git a/docs/cmd_CN.md b/docs/cmd_CN.md
index cd44220..59136fc 100644
--- a/docs/cmd_CN.md
+++ b/docs/cmd_CN.md
@@ -243,6 +243,71 @@ query result pending nonce
 query result: 2
 ```
 
+
+##KeyValue: 通过key查询value 
+
+##### 命令
+
+```
+ gtool  --backend <validator's IP:RPC Port>  get <key>
+```
+
+##### 结果
+
+```
+query result value
+```
+
+##### Demo
+
+```
+./build/gtool --backend "tcp://127.0.0.1:46657" get test_key_0
+query result: v_015
+```
+
+
+## KeyValue : 写入数据
+
+
+##### 命令
+
+```
+gtool --backend <validator's IP:RPC Port> put <key> <value> --priv_key=<priv_key>
+```
+
+##### 结果
+
+```
+tx result 
+```
+
+##### Demo
+
+```
+gtool --backend "tcp://127.0.0.1:46657" put foo bar --priv_key=B37D1C91C2320911EFA1433EC8BBBE48AF15409743F8CB1830708CBBA0DB6690									
+tx result: 0x8eaf236bf0d760c15e798161de84a11c8ed06e7b3622653d85f9dabbd7a44609
+```
+
+## KeyValue  查询更新历史
+
+####Command 
+```
+ gtool  --backend <validator's IP:RPC Port>  get <key> --page_num=<page_num> --page_size=<page_size>
+```
+默认 page_size 为 10, 最大page_size为20
+##### 结果
+
+```
+query result key_update_history
+```
+
+##### Demo
+
+```
+gtool get foo --page_num=1
+query result: {"key":"Zm9v","value_update_histories":[{"tx_hash":"0sWVAv8hCoMp7qcwsdJ57yOAps2zbFFARAa6H3Ad6ko=","block_height":50283,"time_stamp":1588149308,"value":"YmFyMg==","tx_index":0},{"tx_hash":"3gmcnXK6yQvW4oNbZhV0kEVJtZ/GKIsgP/lcRFmO3aI=","block_height":50273,"time_stamp":1588149294,"value":"YmFyMQ==","tx_index":0}],"total":3}
+```
+
 ## 查询 Receipt
 
 ##### 命令
diff --git a/gemmill/types/kv.go b/gemmill/types/kv.go
new file mode 100644
index 0000000..51456ed
--- /dev/null
+++ b/gemmill/types/kv.go
@@ -0,0 +1,54 @@
+package types
+
+import (
+	"encoding/hex"
+	"fmt"
+)
+
+type ValueUpdateHistory struct {
+	TxHash      []byte `json:"tx_hash"`
+	BlockHeight uint64 `json:"block_height"`
+	TimeStamp   uint64 `json:"time_stamp"`
+	Value       []byte `json:"value"`
+	TxIndex     uint32  `json:"tx_index"`
+}
+
+type ValueHistoryResult struct {
+	Key                  []byte                `json:"key"`
+	ValueUpdateHistories []*ValueUpdateHistory `json:"value_update_histories"`
+	Total                uint32                `json:"total"`
+}
+
+func (v *ValueUpdateHistory) String() string {
+	if v == nil {
+		return ""
+	}
+	return fmt.Sprintf("hight : %d , hash :%s ,time: %d , value %s , txIndx %d",
+		v.BlockHeight, hex.EncodeToString(v.TxHash), v.TimeStamp, string(v.Value),v.TxIndex)
+}
+
+type KeyValueHistory struct {
+	Key                []byte
+	ValueUpdateHistory *ValueUpdateHistory
+}
+
+func (k *KeyValueHistory)String()string {
+	if k == nil {
+		return ""
+	}
+	return fmt.Sprintf("key : %s ,history: %s",string(k.Key),k.ValueUpdateHistory.String())
+}
+
+type KeyValueHistories []*KeyValueHistory
+
+func (k KeyValueHistories)String()string {
+	if len(k) ==0 {
+		return ""
+	}
+	result := "["
+	for _, v := range k {
+		result += v.String()+" "
+	}
+	result+="]"
+	return result
+}
\ No newline at end of file
diff --git a/go.mod b/go.mod
index 86794d3..0b3e335 100644
--- a/go.mod
+++ b/go.mod
@@ -6,7 +6,6 @@ require (
 	github.com/BurntSushi/toml v0.3.1
 	github.com/allegro/bigcache v1.2.0
 	github.com/aristanetworks/goarista v0.0.0-20180424004133-70dca2f27708
-	github.com/astaxie/beego v0.0.0-20171218111859-f16688817aa4
 	github.com/bitly/go-simplejson v0.0.0-20170206154632-da1a8928f709
 	github.com/btcsuite/btcd v0.0.0-20190427004231-96897255fd17
 	github.com/davecgh/go-spew v1.1.1