diff --git a/apps/nsq_data_tool/tool.go b/apps/nsq_data_tool/tool.go index dc402f00..a9c4d1b9 100644 --- a/apps/nsq_data_tool/tool.go +++ b/apps/nsq_data_tool/tool.go @@ -79,7 +79,7 @@ func checkTopicStats() { log.Printf("error: %v", err.Error()) return } - producers, _, err := ci.GetNSQDStats(srcnodes, "", "", true) + producers, _, err := ci.GetNSQDStatsWithClients(srcnodes, "", "", true) if err != nil { log.Printf("error: %v", err.Error()) return diff --git a/apps/nsqd/nsqd.go b/apps/nsqd/nsqd.go index fab371e6..2c317bde 100644 --- a/apps/nsqd/nsqd.go +++ b/apps/nsqd/nsqd.go @@ -182,6 +182,11 @@ func nsqdFlagSet(opts *nsqd.Options) *flag.FlagSet { flagSet.Int("queue-write-buffer-size", int(opts.QueueWriteBufferSize), "the write buffer size for topic disk queue file") flagSet.Int("pub-queue-size", int(opts.PubQueueSize), "the pub queue size for topic") flagSet.Int("sleepms-between-log-sync-pull", int(opts.SleepMsBetweenLogSyncPull), "the sleep ms between each log sync pull") + + flagSet.Bool("kv-enabled", opts.KVEnabled, "enable the kv topic") + flagSet.Int("kv-block-cache", int(opts.KVBlockCache), "kv engine block cache") + flagSet.Int("kv-write-buffer-size", int(opts.KVWriteBufferSize), "kv engine write buffer size") + flagSet.Int("kv-max-write-buffer-number", int(opts.KVMaxWriteBufferNumber), "kv max write buffer number") return flagSet } diff --git a/bench/multi_bench/multi_bench.go b/bench/multi_bench/multi_bench.go index 4aa6fc50..b34eb929 100644 --- a/bench/multi_bench/multi_bench.go +++ b/bench/multi_bench/multi_bench.go @@ -17,7 +17,7 @@ import ( "time" "github.com/absolute8511/glog" - "github.com/spaolacci/murmur3" + "github.com/twmb/murmur3" "github.com/youzan/go-nsq" "github.com/youzan/nsq/internal/app" "github.com/youzan/nsq/internal/clusterinfo" diff --git a/consistence/commitlog.go b/consistence/commitlog.go index 6a00a63c..a5ef5d88 100644 --- a/consistence/commitlog.go +++ b/consistence/commitlog.go @@ -79,6 +79,13 @@ type CommitLogData struct { MsgNum int32 } +func (cl *CommitLogData) dqSeekCnt() int64 { + if cl.MsgCnt > 0 { + return cl.MsgCnt - 1 + } + return 0 +} + func GetLogDataSize() int { return logDataSize } @@ -758,6 +765,35 @@ func (self *TopicCommitLogMgr) GetCurrentEnd() (int64, int64) { return self.currentStart, int64(self.currentCount) * int64(GetLogDataSize()) } +func (self *TopicCommitLogMgr) GetMaxAvailableCleanOffset(fileIndex int64, fileOffset int64) (int64, error) { + self.Lock() + defer self.Unlock() + if fileIndex == self.currentStart { + if fileOffset/int64(GetLogDataSize())+int64(MIN_KEEP_LOG_ITEM) >= int64(self.currentCount) { + if int64(self.currentCount) <= int64(MIN_KEEP_LOG_ITEM) { + return fileOffset, ErrCommitLogCleanKeepMin + } else { + return (int64(self.currentCount-1) - int64(MIN_KEEP_LOG_ITEM)) * int64(GetLogDataSize()), nil + } + } + } else if fileIndex == self.currentStart-1 { + fName := getSegmentFilename(self.path, fileIndex) + stat, err := os.Stat(fName) + if err != nil { + return fileOffset, err + } + leftSize := stat.Size() - fileOffset + if leftSize < 0 { + return fileOffset, ErrCommitLogOffsetInvalid + } + if leftSize/int64(GetLogDataSize())+int64(self.currentCount) <= int64(MIN_KEEP_LOG_ITEM) { + noffset := stat.Size() - (int64(MIN_KEEP_LOG_ITEM)-int64(self.currentCount)+1)*int64(GetLogDataSize()) + return noffset, nil + } + } + return fileOffset, nil +} + func (self *TopicCommitLogMgr) prepareCleanOldData(fileIndex int64, fileOffset int64) (LogStartInfo, int64, error) { self.Lock() defer self.Unlock() diff --git a/consistence/data_placement_mgr.go b/consistence/data_placement_mgr.go index 0e9268c7..c754c8a0 100644 --- a/consistence/data_placement_mgr.go +++ b/consistence/data_placement_mgr.go @@ -11,7 +11,7 @@ import ( "sync/atomic" "time" - "github.com/spaolacci/murmur3" + "github.com/twmb/murmur3" "github.com/youzan/nsq/nsqd" ) diff --git a/consistence/etcd_client.go b/consistence/etcd_client.go index 4acaf001..79bfb19f 100644 --- a/consistence/etcd_client.go +++ b/consistence/etcd_client.go @@ -19,8 +19,8 @@ type EtcdClient struct { var etcdTransport client.CancelableTransport = &http.Transport{ Proxy: http.ProxyFromEnvironment, Dial: (&net.Dialer{ - Timeout: 30 * time.Second, - KeepAlive: 15 * time.Second, + Timeout: 10 * time.Second, + KeepAlive: 10 * time.Second, }).Dial, TLSHandshakeTimeout: 10 * time.Second, WriteBufferSize: 1024, diff --git a/consistence/leadership.go b/consistence/leadership.go index b7b15686..83511de8 100644 --- a/consistence/leadership.go +++ b/consistence/leadership.go @@ -58,7 +58,7 @@ type NsqLookupdNodeInfo struct { Epoch EpochType } -func (self *NsqLookupdNodeInfo) GetID() string { +func (self NsqLookupdNodeInfo) GetID() string { return self.ID } diff --git a/consistence/nsqd_coordinator.go b/consistence/nsqd_coordinator.go index 877bd987..dd8d8a40 100644 --- a/consistence/nsqd_coordinator.go +++ b/consistence/nsqd_coordinator.go @@ -72,10 +72,11 @@ type ILocalLogQueue interface { ForceFlush() ResetBackendEndNoLock(nsqd.BackendOffset, int64) error ResetBackendWithQueueStartNoLock(int64, int64) error - GetDiskQueueSnapshot() *nsqd.DiskQueueSnapshot + GetDiskQueueSnapshot(checkCommit bool) *nsqd.DiskQueueSnapshot TotalMessageCnt() uint64 TotalDataSize() int64 TryFixQueueEnd(nsqd.BackendOffset, int64) error + CheckDiskQueueReadToEndOK(offset int64, seekCnt int64, endOffset nsqd.BackendOffset) error PutRawDataOnReplica(rawData []byte, offset nsqd.BackendOffset, checkSize int64, msgNum int32) (nsqd.BackendQueueEnd, error) PutMessageOnReplica(msgs *nsqd.Message, offset nsqd.BackendOffset, checkSize int64) (nsqd.BackendQueueEnd, error) @@ -377,6 +378,27 @@ func doLogQClean(tcData *coordData, localTopic *nsqd.Topic, retentionSize int64, // so we should not clean the segment at the middle of the batch. maxCleanOffset = nsqd.BackendOffset(l.MsgOffset) } + oldMatchOffset := matchOffset + matchOffset, err = logMgr.GetMaxAvailableCleanOffset(matchIndex, matchOffset) + if err != nil { + coordLog.Infof("clean commit log no more available clean at (%v-%v): %s", matchIndex, matchOffset, err) + return + } + if matchOffset >= oldMatchOffset { + // should not exceed last check + matchOffset = oldMatchOffset + } else { + // relocate the offset for the topic data + l, err = logMgr.GetCommitLogFromOffsetV2(matchIndex, matchOffset) + if err != nil { + coordLog.Infof("clean commit log failed at (%v-%v): %s", matchIndex, matchOffset, err) + return + } + if l.MsgOffset < int64(maxCleanOffset) { + maxCleanOffset = nsqd.BackendOffset(l.MsgOffset) + } + } + coordLog.Infof("clean commit log relocated at : %v, %v, %v, %v", matchIndex, matchOffset, maxCleanOffset, l) err = logMgr.CleanOldData(matchIndex, matchOffset) if err != nil { coordLog.Infof("clean commit log err : %v", err) @@ -592,7 +614,7 @@ func (ncoord *NsqdCoordinator) watchNsqLookupd() { // should not hold any coordinator lock outside, since we may wait local topic pub loop quit. // pub loop may be blocked by coordinator lock. func (ncoord *NsqdCoordinator) checkLocalTopicMagicCode(topicInfo *TopicPartitionMetaInfo, tryFix bool) error { - removedPath, err := ncoord.localNsqd.CheckMagicCode(topicInfo.Name, topicInfo.Partition, topicInfo.MagicCode, tryFix) + removedPath, err := ncoord.localNsqd.CheckMagicCode(topicInfo.Name, topicInfo.Partition, topicInfo.MagicCode, topicInfo.Ext, tryFix) if err != nil { coordLog.Infof("check magic code error: %v", err) return err @@ -842,6 +864,7 @@ func (ncoord *NsqdCoordinator) loadLocalTopicData() error { maybeInitDelayedQ(tc.GetData(), topic) topic.SetDynamicInfo(*dyConf, tc.GetData().logMgr) + // do we need hold lock for local topic? since it will not run concurrently with catchup and will run in single goroutine localErr := checkAndFixLocalLogQueueData(tc.GetData(), topic, tc.GetData().logMgr, forceFix) if localErr != nil { coordLog.Errorf("check local topic %v data need to be fixed:%v", topicInfo.GetTopicDesp(), localErr) @@ -906,7 +929,12 @@ func checkAndFixLocalLogQueueEnd(tc *coordData, commitEndOffset := nsqd.BackendOffset(logData.MsgOffset + int64(logData.MsgSize)) commitEndCnt := logData.MsgCnt + int64(logData.MsgNum) - 1 if localLogQ.TotalDataSize() == int64(commitEndOffset) && localLogQ.TotalMessageCnt() == uint64(commitEndCnt) { - return nil + // need read one data to ensure the data can be read ok (In some case, such as disk full, it may have zero data padding unexpected) + if localErr := localLogQ.CheckDiskQueueReadToEndOK(logData.MsgOffset, logData.dqSeekCnt(), commitEndOffset); localErr == nil { + return nil + } else { + coordLog.Errorf("topic %v need fix, read end failed: %v , %v:%v", tname, localErr, logIndex, logOffset) + } } coordLog.Infof("current topic %v log: %v:%v, %v, diskqueue end: %v", tname, logIndex, logOffset, logData, localLogQ.TotalDataSize()) @@ -916,7 +944,13 @@ func checkAndFixLocalLogQueueEnd(tc *coordData, } localErr := localLogQ.ResetBackendEndNoLock(commitEndOffset, commitEndCnt) if localErr == nil { - return nil + // need read one data to ensure the data can be read ok (In some case, such as disk full, it may have zero data padding unexpected) + localErr = localLogQ.CheckDiskQueueReadToEndOK(logData.MsgOffset, logData.dqSeekCnt(), commitEndOffset) + if localErr == nil { + return nil + } else { + coordLog.Errorf("topic %v need fix, read end failed: %v , %v:%v", tname, localErr, logIndex, logOffset) + } } coordLog.Errorf("topic %v reset local queue backend failed: %v", tname, localErr) if !forceFix { @@ -940,8 +974,9 @@ func checkAndFixLocalLogQueueEnd(tc *coordData, coordLog.Errorf("topic %v try fix failed: %v , %v:%v", tname, localErr, logIndex, logOffset) break } - if logData.MsgOffset+int64(logData.MsgSize) <= realEnd { - localErr = localLogQ.ResetBackendEndNoLock(nsqd.BackendOffset(logData.MsgOffset+int64(logData.MsgSize)), + endOffset := nsqd.BackendOffset(logData.MsgOffset + int64(logData.MsgSize)) + if int64(endOffset) <= realEnd { + localErr = localLogQ.ResetBackendEndNoLock(endOffset, logData.MsgCnt+int64(logData.MsgNum)-1) if localErr != nil { coordLog.Infof("topic %v reset local queue failed: %v at %v:%v", tname, localErr, logIndex, logOffset) @@ -953,29 +988,25 @@ func checkAndFixLocalLogQueueEnd(tc *coordData, coordLog.Errorf("topic %v reset local queue failed: %v, at %v:%v", tname, localErr, logIndex, logOffset) } else { - return nil + // need read one data to ensure the data can be read ok (In some case, such as disk full, it may have zero data padding unexpected) + localErr = localLogQ.CheckDiskQueueReadToEndOK(logData.MsgOffset, logData.dqSeekCnt(), endOffset) + if localErr == nil { + return nil + } else { + coordLog.Errorf("topic %v need fix, read end failed: %v , %v:%v", tname, localErr, logIndex, logOffset) + } } } } else { - localErr = localLogQ.TryFixQueueEnd(nsqd.BackendOffset(logData.MsgOffset+int64(logData.MsgSize)), + localErr = localLogQ.TryFixQueueEnd(endOffset, logData.MsgCnt+int64(logData.MsgNum)-1) if localErr != nil { continue } - snap := localLogQ.GetDiskQueueSnapshot() - seekCnt := int64(0) - if logData.MsgCnt > 0 { - seekCnt = logData.MsgCnt - 1 - } - localErr = snap.SeekTo(nsqd.BackendOffset(logData.MsgOffset), seekCnt) + localErr = localLogQ.CheckDiskQueueReadToEndOK(logData.MsgOffset, logData.dqSeekCnt(), endOffset) if localErr != nil { - coordLog.Errorf("topic %v try fix end failed: %v , %v:%v", tname, localErr, logIndex, logOffset) - break - } - r := snap.ReadOne() - if r.Err != nil { - coordLog.Errorf("topic %v try fix end failed: %v , %v:%v", tname, r.Err, logIndex, logOffset) - break + coordLog.Errorf("topic %v try fix end, read failed: %v , %v:%v", tname, localErr, logIndex, logOffset) + continue } return nil } @@ -998,12 +1029,9 @@ func checkAndFixLocalLogQueueData(tc *coordData, } endFixErr := checkAndFixLocalLogQueueEnd(tc, localLogQ, logMgr, true, forceFix) - snap := localLogQ.GetDiskQueueSnapshot() + snap := localLogQ.GetDiskQueueSnapshot(false) for { - seekCnt := int64(0) - if log.MsgCnt > 0 { - seekCnt = log.MsgCnt - 1 - } + seekCnt := log.dqSeekCnt() err = snap.SeekTo(nsqd.BackendOffset(log.MsgOffset), seekCnt) if err == nil { break @@ -1250,7 +1278,7 @@ func (ncoord *NsqdCoordinator) SearchLogByMsgID(topic string, part int, msgID in return nil, 0, 0, localErr } realOffset := l.MsgOffset - curCount := l.MsgCnt - 1 + curCount := l.dqSeekCnt() return l, realOffset, curCount, nil } @@ -1266,18 +1294,15 @@ func (ncoord *NsqdCoordinator) SearchLogByMsgOffset(topic string, part int, offs return nil, 0, 0, localErr } realOffset := l.MsgOffset - curCount := l.MsgCnt - 1 + curCount := l.dqSeekCnt() if l.MsgOffset < offset { t, localErr := ncoord.localNsqd.GetExistingTopic(topic, part) if localErr != nil { return l, 0, 0, localErr } - snap := t.GetDiskQueueSnapshot() + snap := t.GetDiskQueueSnapshot(true) - seekCnt := int64(0) - if l.MsgCnt > 0 { - seekCnt = l.MsgCnt - 1 - } + seekCnt := l.dqSeekCnt() localErr = snap.SeekTo(nsqd.BackendOffset(realOffset), seekCnt) if localErr != nil { coordLog.Infof("seek to disk queue error: %v, %v", localErr, realOffset) @@ -1315,18 +1340,15 @@ func (ncoord *NsqdCoordinator) SearchLogByMsgCnt(topic string, part int, count i return nil, 0, 0, localErr } realOffset := l.MsgOffset - curCount := l.MsgCnt - 1 + curCount := l.dqSeekCnt() if l.MsgCnt < count { t, localErr := ncoord.localNsqd.GetExistingTopic(topic, part) if localErr != nil { return l, 0, 0, localErr } - snap := t.GetDiskQueueSnapshot() + snap := t.GetDiskQueueSnapshot(true) - seekCnt := int64(0) - if l.MsgCnt > 0 { - seekCnt = l.MsgCnt - 1 - } + seekCnt := l.dqSeekCnt() localErr = snap.SeekTo(nsqd.BackendOffset(realOffset), seekCnt) if localErr != nil { coordLog.Infof("seek to disk queue error: %v, %v", localErr, realOffset) @@ -1365,11 +1387,7 @@ func (ncoord *MsgTimestampComparator) SearchEndBoundary() int64 { } func (ncoord *MsgTimestampComparator) LessThanLeftBoundary(l *CommitLogData) bool { - - seekCnt := int64(0) - if l.MsgCnt > 0 { - seekCnt = l.MsgCnt - 1 - } + seekCnt := l.dqSeekCnt() err := ncoord.localTopicReader.ResetSeekTo(nsqd.BackendOffset(l.MsgOffset), seekCnt) if err != nil { coordLog.Errorf("seek disk queue failed: %v, %v", l, err) @@ -1392,10 +1410,7 @@ func (ncoord *MsgTimestampComparator) LessThanLeftBoundary(l *CommitLogData) boo } func (ncoord *MsgTimestampComparator) GreatThanRightBoundary(l *CommitLogData) bool { - seekCnt := int64(0) - if l.MsgCnt > 0 { - seekCnt = l.MsgCnt - 1 - } + seekCnt := l.dqSeekCnt() // we may read the eof , in this situation we reach the end, so the search should not be great than right boundary err := ncoord.localTopicReader.ResetSeekTo(nsqd.BackendOffset(l.MsgOffset+int64(l.MsgSize)), seekCnt) if err != nil { @@ -1429,7 +1444,7 @@ func (ncoord *NsqdCoordinator) SearchLogByMsgTimestamp(topic string, part int, t return nil, 0, 0, localErr } - snap := t.GetDiskQueueSnapshot() + snap := t.GetDiskQueueSnapshot(true) comp := &MsgTimestampComparator{ localTopicReader: snap, searchEnd: tcData.logMgr.GetCurrentStart(), @@ -1449,17 +1464,14 @@ func (ncoord *NsqdCoordinator) SearchLogByMsgTimestamp(topic string, part int, t } realOffset := l.MsgOffset - seekCnt := int64(0) - if l.MsgCnt > 0 { - seekCnt = l.MsgCnt - 1 - } + seekCnt := l.dqSeekCnt() // check if the message timestamp is fit the require localErr = snap.ResetSeekTo(nsqd.BackendOffset(realOffset), seekCnt) if localErr != nil { coordLog.Infof("seek to disk queue error: %v, %v", localErr, realOffset) return l, 0, 0, localErr } - curCount := l.MsgCnt - 1 + curCount := l.dqSeekCnt() for { ret := snap.ReadOne() @@ -1676,6 +1688,7 @@ func (ncoord *NsqdCoordinator) decideCatchupCommitLogInfo(tc *TopicCoordinator, } needFullSync = true } + // do we need hold lock for local topic? Maybe no since it will not run concurrently while catchup localErr := checkAndFixLocalLogQueueData(tc.GetData(), localLogQ, logMgr, false) if localErr != nil { coordLog.Errorf("check local topic %v data need to be fixed:%v", topicInfo.GetTopicDesp(), localErr) @@ -2867,13 +2880,13 @@ func (ncoord *NsqdCoordinator) readTopicRawData(topic string, partition int, off return nil, ErrLocalTopicPartitionMismatch } dataList := make([][]byte, 0, len(offsetList)) - snap := t.GetDiskQueueSnapshot() + snap := t.GetDiskQueueSnapshot(true) if fromDelayedQueue { dq := t.GetDelayedQueue() if dq == nil { return nil, ErrLocalDelayedQueueMissing } - snap = dq.GetDiskQueueSnapshot() + snap = dq.GetDiskQueueSnapshot(true) } for i, offset := range offsetList { size := sizeList[i] @@ -3005,7 +3018,9 @@ func (ncoord *NsqdCoordinator) updateLocalTopic(topicInfo *TopicPartitionMetaInf } if t.IsDataNeedFix() { + t.Lock() endFixErr := checkAndFixLocalLogQueueEnd(tcData, t, tcData.logMgr, true, ForceFixLeaderData) + t.Unlock() if endFixErr != nil { t.SetDataFixState(true) } diff --git a/consistence/nsqd_coordinator_cluster_write.go b/consistence/nsqd_coordinator_cluster_write.go index 4984e599..dee950c4 100644 --- a/consistence/nsqd_coordinator_cluster_write.go +++ b/consistence/nsqd_coordinator_cluster_write.go @@ -368,7 +368,7 @@ func (ncoord *NsqdCoordinator) doSyncOpToCluster(isWrite bool, coord *TopicCoord topicFullName := topicName + strconv.Itoa(topicPartition) var clusterWriteErr *CoordErr - if clusterWriteErr = tcData.checkWriteForLeader(ncoord.myNode.GetID()); clusterWriteErr != nil { + if clusterWriteErr = coord.checkWriteForLeader(ncoord.myNode.GetID()); clusterWriteErr != nil { coordLog.Warningf("topic(%v) check write failed :%v", topicFullName, clusterWriteErr) coordErrStats.incWriteErr(clusterWriteErr) return clusterWriteErr @@ -421,7 +421,7 @@ retrysync: } if needRefreshISR { tcData = coord.GetData() - if clusterWriteErr = tcData.checkWriteForLeader(ncoord.myNode.GetID()); clusterWriteErr != nil { + if clusterWriteErr = coord.checkWriteForLeader(ncoord.myNode.GetID()); clusterWriteErr != nil { coordLog.Warningf("topic(%v) check operation failed :%v", topicFullName, clusterWriteErr) goto exitsync } @@ -1070,7 +1070,11 @@ func (ncoord *NsqdCoordinator) UpdateChannelStateToCluster(channel *nsqd.Channel return rpcErr } handleSyncResult := func(successNum int, tcData *coordData) bool { - return true + // make sure the state sync failed can be known for api caller + if successNum == len(tcData.topicInfo.ISR) { + return true + } + return false } clusterErr := ncoord.doSyncOpToCluster(false, coord, doLocalWrite, doLocalExit, doLocalCommit, doLocalRollback, doRefresh, doSlaveSync, handleSyncResult) @@ -1319,11 +1323,14 @@ func (ncoord *NsqdCoordinator) DeleteChannel(topic *nsqd.Topic, channelName stri } coord.GetData().syncedConsumeMgr.Clear() + var leaderErr error doLocalWrite := func(d *coordData) *CoordErr { localErr := topic.DeleteExistingChannel(channelName) + // we continue try to delete on the follower even leader failed since it can be auto-created by client. if localErr != nil { coordLog.Infof("topic %v deleteing local channel %v error: %v", topicName, channelName, localErr) + leaderErr = localErr } return nil } @@ -1347,7 +1354,6 @@ func (ncoord *NsqdCoordinator) DeleteChannel(topic *nsqd.Topic, channelName stri return rpcErr } handleSyncResult := func(successNum int, tcData *coordData) bool { - // we can ignore the error if this channel is not ordered. (just sync next time) if successNum == len(tcData.topicInfo.ISR) { return true } @@ -1358,6 +1364,10 @@ func (ncoord *NsqdCoordinator) DeleteChannel(topic *nsqd.Topic, channelName stri if clusterErr != nil { return clusterErr.ToErrorType() } + if leaderErr != nil { + // leader failed should also be notified to caller + return leaderErr + } return nil } @@ -1379,6 +1389,8 @@ func (ncoord *NsqdCoordinator) deleteChannelOnSlave(tc *coordData, channelName s localErr = topic.DeleteExistingChannel(channelName) if localErr != nil { coordLog.Logf("topic %v delete channel %v on slave failed: %v ", topicName, channelName, localErr) + // since delete not exist will have no error, err should be returned to caller + return &CoordErr{localErr.Error(), RpcNoErr, CoordLocalErr} } else { tc.syncedConsumeMgr.Clear() } diff --git a/consistence/nsqd_coordinator_test.go b/consistence/nsqd_coordinator_test.go index dc88f16b..77193a71 100644 --- a/consistence/nsqd_coordinator_test.go +++ b/consistence/nsqd_coordinator_test.go @@ -66,6 +66,7 @@ type testClusterNodeInfo struct { } type fakeLookupRemoteProxy struct { + mu sync.Mutex leaderSessions map[string]map[int]*TopicLeaderSession fakeNsqdCoords map[string]*NsqdCoordinator lookupEpoch EpochType @@ -81,6 +82,12 @@ func NewFakeLookupRemoteProxy(addr string, timeout time.Duration) (INsqlookupRem }, nil } +func (self *fakeLookupRemoteProxy) changeNodeCoord(id string, coord *NsqdCoordinator) { + self.mu.Lock() + defer self.mu.Unlock() + self.fakeNsqdCoords[id] = coord +} + func (self *fakeLookupRemoteProxy) RemoteAddr() string { return self.addr } @@ -118,6 +125,8 @@ func (self *fakeLookupRemoteProxy) RequestJoinTopicISR(topic string, partition i self.t.Log("requesting join isr") } // notify disable topic write + self.mu.Lock() + defer self.mu.Unlock() for nid, nsqdCoord := range self.fakeNsqdCoords { tp, err := nsqdCoord.getTopicCoord(topic, partition) if err != nil { @@ -139,6 +148,8 @@ func (self *fakeLookupRemoteProxy) ReadyForTopicISR(topic string, partition int, if self.t != nil { self.t.Log("requesting ready for isr") } + self.mu.Lock() + defer self.mu.Unlock() localSession, ok := self.leaderSessions[topic] if !ok { return ErrMissingTopicLeaderSession @@ -178,6 +189,8 @@ func (self *fakeLookupRemoteProxy) RequestLeaveFromISRByLeader(topic string, par if self.t != nil { self.t.Log("requesting leave isr by leader") } + self.mu.Lock() + defer self.mu.Unlock() if self.leaderSessions[topic][partition].IsSame(leaderSession) { return nil } @@ -195,7 +208,7 @@ func mustStartNSQD(opts *nsqdNs.Options) *nsqdNs.NSQD { } opts.DataPath = tmpDir } - nsqd := nsqdNs.New(opts) + nsqd, _ := nsqdNs.New(opts) return nsqd } @@ -1467,7 +1480,7 @@ func TestNsqdCoordCatchupWhileWriting(t *testing.T) { test.Nil(t, err) test.Equal(t, len(logs1), int(msgCnt)) // add to fake lookup to notify flush data while join isr notify - fakeLookupProxy.(*fakeLookupRemoteProxy).fakeNsqdCoords[nsqdCoord1.GetMyID()] = nsqdCoord1 + fakeLookupProxy.(*fakeLookupRemoteProxy).changeNodeCoord(nsqdCoord1.GetMyID(), nsqdCoord1) // the follower should notify join isr while reached EOF error, // and the faked lookup will flush the left disk queue data to make no EOF error, and need retry for catchup topicData3.ForceFlush() @@ -2251,6 +2264,12 @@ func TestChannelCreateSync(t *testing.T) { test.Equal(t, c1, t3ch1.GetConfirmed()) } +func TestNsqdCoordFixTopicCorruptEnd(t *testing.T) { + // fix start + // fix end + // fix end zero padding (may happend while disk full) +} + func benchmarkNsqdCoordPubWithArg(b *testing.B, replica int, size int, useExt bool) { b.StopTimer() topicBase := "coordBenchTestTopic" diff --git a/consistence/nsqd_node_etcd.go b/consistence/nsqd_node_etcd.go index 2abcaa10..0906cda9 100644 --- a/consistence/nsqd_node_etcd.go +++ b/consistence/nsqd_node_etcd.go @@ -12,8 +12,7 @@ import ( ) const ( - ETCD_TTL = 20 - watchEtcdTimeout = time.Second * time.Duration(ETCD_TTL*10) + ETCD_TTL = 20 ) type MasterChanInfo struct { diff --git a/consistence/nsqd_rpc_client.go b/consistence/nsqd_rpc_client.go index b7e60b91..1054b0e6 100644 --- a/consistence/nsqd_rpc_client.go +++ b/consistence/nsqd_rpc_client.go @@ -162,10 +162,10 @@ func (nrpc *NsqdRpcClient) CallWithRetry(method string, arg interface{}) (interf reply, err = nrpc.dc.Call(method, arg) if testRPCTimeoutAndWait { time.Sleep(maxWriteWaitTimeout) - e := gorpc.ErrCanceled + e := *gorpc.ErrCanceled e.Timeout = true e.Connection = true - err = e + err = &e } if err != nil { cerr, ok := err.(*gorpc.ClientError) diff --git a/consistence/nsqd_rpc_client_test.go b/consistence/nsqd_rpc_client_test.go index 1b9d6a0a..e3db931e 100644 --- a/consistence/nsqd_rpc_client_test.go +++ b/consistence/nsqd_rpc_client_test.go @@ -227,7 +227,9 @@ func startNsqdCoordWithFakeData(t *testing.T, rpcport string, dataPath string, nsqdCoord.leadership = fakeLeadership nsqdCoord.lookupRemoteCreateFunc = func(addr string, to time.Duration) (INsqlookupRemoteProxy, error) { fakeLookupProxy.t = t + fakeLookupProxy.mu.Lock() fakeLookupProxy.fakeNsqdCoords[nsqdCoord.myNode.GetID()] = nsqdCoord + fakeLookupProxy.mu.Unlock() return fakeLookupProxy, nil } nsqdCoord.lookupLeader = NsqLookupdNodeInfo{} diff --git a/consistence/nsqlookup_coord_api.go b/consistence/nsqlookup_coord_api.go index 0c2dda5d..e3ea6bdb 100644 --- a/consistence/nsqlookup_coord_api.go +++ b/consistence/nsqlookup_coord_api.go @@ -22,6 +22,8 @@ func (nlcoord *NsqLookupCoordinator) GetAllLookupdNodes() ([]NsqLookupdNodeInfo, } func (nlcoord *NsqLookupCoordinator) GetLookupLeader() NsqLookupdNodeInfo { + nlcoord.leaderMu.RLock() + defer nlcoord.leaderMu.RUnlock() return nlcoord.leaderNode } @@ -82,7 +84,7 @@ func (nlcoord *NsqLookupCoordinator) GetTopicLeaderForConsume(topicName string, } func (nlcoord *NsqLookupCoordinator) IsMineLeader() bool { - return nlcoord.leaderNode.GetID() == nlcoord.myNode.GetID() + return nlcoord.GetLookupLeader().GetID() == nlcoord.myNode.GetID() } func (nlcoord *NsqLookupCoordinator) IsClusterStable() bool { @@ -91,7 +93,7 @@ func (nlcoord *NsqLookupCoordinator) IsClusterStable() bool { } func (nlcoord *NsqLookupCoordinator) SetTopNBalance(enable bool) error { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { coordLog.Infof("not leader while delete topic") return ErrNotNsqLookupLeader } @@ -104,7 +106,7 @@ func (nlcoord *NsqLookupCoordinator) SetTopNBalance(enable bool) error { } func (nlcoord *NsqLookupCoordinator) SetClusterUpgradeState(upgrading bool) error { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { coordLog.Infof("not leader while delete topic") return ErrNotNsqLookupLeader } @@ -201,7 +203,7 @@ func (nlcoord *NsqLookupCoordinator) IsTopicLeader(topic string, part int, nid s } func (nlcoord *NsqLookupCoordinator) MarkNodeAsRemoving(nid string) error { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { coordLog.Infof("not leader while delete topic") return ErrNotNsqLookupLeader } @@ -223,7 +225,7 @@ func (nlcoord *NsqLookupCoordinator) MarkNodeAsRemoving(nid string) error { } func (nlcoord *NsqLookupCoordinator) DeleteTopicForce(topic string, partition string) error { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { coordLog.Infof("not leader while delete topic") return ErrNotNsqLookupLeader } @@ -272,7 +274,7 @@ func (nlcoord *NsqLookupCoordinator) DeleteTopicForce(topic string, partition st } func (nlcoord *NsqLookupCoordinator) DeleteTopic(topic string, partition string) error { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { coordLog.Infof("not leader while delete topic") return ErrNotNsqLookupLeader } @@ -350,7 +352,7 @@ func (nlcoord *NsqLookupCoordinator) deleteTopicPartitionForce(topic string, pid coordLog.Infof("failed to get rpc client: %v, %v", node.ID, rpcErr) continue } - rpcErr = c.DeleteNsqdTopic(nlcoord.leaderNode.Epoch, &topicInfo) + rpcErr = c.DeleteNsqdTopic(nlcoord.GetLookupLeader().Epoch, &topicInfo) if rpcErr != nil { coordLog.Infof("failed to call rpc : %v, %v", node.ID, rpcErr) } @@ -375,7 +377,7 @@ func (nlcoord *NsqLookupCoordinator) deleteTopicPartition(topic string, pid int) coordLog.Infof("failed to get rpc client: %v, %v", id, rpcErr) continue } - rpcErr = c.DeleteNsqdTopic(nlcoord.leaderNode.Epoch, topicInfo) + rpcErr = c.DeleteNsqdTopic(nlcoord.GetLookupLeader().Epoch, topicInfo) if rpcErr != nil { coordLog.Infof("failed to call rpc : %v, %v", id, rpcErr) } @@ -386,7 +388,7 @@ func (nlcoord *NsqLookupCoordinator) deleteTopicPartition(topic string, pid int) coordLog.Infof("failed to get rpc client: %v, %v", id, rpcErr) continue } - rpcErr = c.DeleteNsqdTopic(nlcoord.leaderNode.Epoch, topicInfo) + rpcErr = c.DeleteNsqdTopic(nlcoord.GetLookupLeader().Epoch, topicInfo) if rpcErr != nil { coordLog.Infof("failed to call rpc : %v, %v", id, rpcErr) } @@ -398,7 +400,7 @@ func (nlcoord *NsqLookupCoordinator) deleteTopicPartition(topic string, pid int) if rpcErr != nil { continue } - c.DeleteNsqdTopic(nlcoord.leaderNode.Epoch, topicInfo) + c.DeleteNsqdTopic(nlcoord.GetLookupLeader().Epoch, topicInfo) } return nil @@ -435,7 +437,7 @@ func (nlcoord *NsqLookupCoordinator) GetRegisteredChannel(topic string) ([]strin //Update registered channels func (nlcoord *NsqLookupCoordinator) UpdateRegisteredChannel(topic string, registeredChannels []string) error { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { coordLog.Infof("not leader while create topic") return ErrNotNsqLookupLeader } @@ -507,7 +509,7 @@ func (nlcoord *NsqLookupCoordinator) UpdateRegisteredChannel(topic string, regis func (nlcoord *NsqLookupCoordinator) ChangeTopicMetaParam(topic string, newSyncEvery int, newRetentionDay int, newReplicator int, upgradeExt string, disableChannelAutoCreate string) error { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { coordLog.Infof("not leader while create topic") return ErrNotNsqLookupLeader } @@ -645,7 +647,7 @@ func (nlcoord *NsqLookupCoordinator) updateTopicMeta(currentNodes map[string]Nsq } func (nlcoord *NsqLookupCoordinator) ExpandTopicPartition(topic string, newPartitionNum int) error { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { coordLog.Infof("not leader while create topic") return ErrNotNsqLookupLeader } @@ -701,7 +703,7 @@ func (nlcoord *NsqLookupCoordinator) ExpandTopicPartition(topic string, newParti } func (nlcoord *NsqLookupCoordinator) CreateTopic(topic string, meta TopicMetaInfo) error { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { coordLog.Infof("not leader while create topic") return ErrNotNsqLookupLeader } diff --git a/consistence/nsqlookup_coord_rpc_helper.go b/consistence/nsqlookup_coord_rpc_helper.go index 7b818051..55f7971c 100644 --- a/consistence/nsqlookup_coord_rpc_helper.go +++ b/consistence/nsqlookup_coord_rpc_helper.go @@ -45,7 +45,7 @@ func (nlcoord *NsqLookupCoordinator) rpcFailRetryFunc(monitorChan chan struct{}) nlcoord.rpcMutex.Unlock() } - epoch := nlcoord.leaderNode.Epoch + epoch := nlcoord.GetLookupLeader().Epoch for _, info := range failList { // check if exiting select { @@ -162,14 +162,14 @@ func (nlcoord *NsqLookupCoordinator) notifyTopicLeaderSession(topicInfo *TopicPa others := getOthersExceptLeader(topicInfo) coordLog.Infof("notify topic leader session changed: %v, %v, others: %v", topicInfo.GetTopicDesp(), leaderSession.Session, others) err := nlcoord.doNotifyToTopicLeaderThenOthers(false, topicInfo.Leader, others, func(nid string) *CoordErr { - return nlcoord.sendTopicLeaderSessionToNsqd(nlcoord.leaderNode.Epoch, nid, topicInfo, leaderSession, joinSession) + return nlcoord.sendTopicLeaderSessionToNsqd(nlcoord.GetLookupLeader().Epoch, nid, topicInfo, leaderSession, joinSession) }) return err } func (nlcoord *NsqLookupCoordinator) notifyAcquireTopicLeader(topicInfo *TopicPartitionMetaInfo) *CoordErr { rpcErr := nlcoord.doNotifyToSingleNsqdNode(topicInfo.Leader, func(nid string) *CoordErr { - return nlcoord.sendAcquireTopicLeaderToNsqd(nlcoord.leaderNode.Epoch, nid, topicInfo) + return nlcoord.sendAcquireTopicLeaderToNsqd(nlcoord.GetLookupLeader().Epoch, nid, topicInfo) }) if rpcErr != nil { coordLog.Infof("notify leader to acquire leader failed: %v", rpcErr) @@ -180,7 +180,7 @@ func (nlcoord *NsqLookupCoordinator) notifyAcquireTopicLeader(topicInfo *TopicPa func (nlcoord *NsqLookupCoordinator) notifyReleaseTopicLeader(topicInfo *TopicPartitionMetaInfo, leaderSessionEpoch EpochType, leaderSession string) *CoordErr { rpcErr := nlcoord.doNotifyToSingleNsqdNode(topicInfo.Leader, func(nid string) *CoordErr { - return nlcoord.sendReleaseTopicLeaderToNsqd(nlcoord.leaderNode.Epoch, nid, topicInfo, leaderSessionEpoch, leaderSession) + return nlcoord.sendReleaseTopicLeaderToNsqd(nlcoord.GetLookupLeader().Epoch, nid, topicInfo, leaderSessionEpoch, leaderSession) }) if rpcErr != nil { coordLog.Infof("notify leader to acquire leader failed: %v", rpcErr) @@ -190,7 +190,7 @@ func (nlcoord *NsqLookupCoordinator) notifyReleaseTopicLeader(topicInfo *TopicPa func (nlcoord *NsqLookupCoordinator) notifyISRTopicMetaInfo(topicInfo *TopicPartitionMetaInfo) *CoordErr { rpcErr := nlcoord.doNotifyToNsqdNodes(topicInfo.ISR, func(nid string) *CoordErr { - return nlcoord.sendTopicInfoToNsqd(nlcoord.leaderNode.Epoch, nid, topicInfo) + return nlcoord.sendTopicInfoToNsqd(nlcoord.GetLookupLeader().Epoch, nid, topicInfo) }) if rpcErr != nil { coordLog.Infof("notify isr for topic meta info failed: %v", rpcErr) @@ -200,7 +200,7 @@ func (nlcoord *NsqLookupCoordinator) notifyISRTopicMetaInfo(topicInfo *TopicPart func (nlcoord *NsqLookupCoordinator) notifyCatchupTopicMetaInfo(topicInfo *TopicPartitionMetaInfo) *CoordErr { rpcErr := nlcoord.doNotifyToNsqdNodes(topicInfo.CatchupList, func(nid string) *CoordErr { - return nlcoord.sendTopicInfoToNsqd(nlcoord.leaderNode.Epoch, nid, topicInfo) + return nlcoord.sendTopicInfoToNsqd(nlcoord.GetLookupLeader().Epoch, nid, topicInfo) }) if rpcErr != nil { coordLog.Infof("notify catchup for topic meta info failed: %v", rpcErr) @@ -215,7 +215,7 @@ func (nlcoord *NsqLookupCoordinator) notifyTopicMetaInfo(topicInfo *TopicPartiti coordLog.Infof("==== notify topic name is empty") } rpcErr := nlcoord.doNotifyToTopicLeaderThenOthers(false, topicInfo.Leader, others, func(nid string) *CoordErr { - return nlcoord.sendTopicInfoToNsqd(nlcoord.leaderNode.Epoch, nid, topicInfo) + return nlcoord.sendTopicInfoToNsqd(nlcoord.GetLookupLeader().Epoch, nid, topicInfo) }) if rpcErr != nil { coordLog.Infof("notify topic meta info failed: %v", rpcErr) @@ -225,7 +225,7 @@ func (nlcoord *NsqLookupCoordinator) notifyTopicMetaInfo(topicInfo *TopicPartiti func (nlcoord *NsqLookupCoordinator) notifyOldNsqdsForTopicMetaInfo(topicInfo *TopicPartitionMetaInfo, oldNodes []string) *CoordErr { return nlcoord.doNotifyToNsqdNodes(oldNodes, func(nid string) *CoordErr { - return nlcoord.sendTopicInfoToNsqd(nlcoord.leaderNode.Epoch, nid, topicInfo) + return nlcoord.sendTopicInfoToNsqd(nlcoord.GetLookupLeader().Epoch, nid, topicInfo) }) } @@ -299,7 +299,7 @@ func (nlcoord *NsqLookupCoordinator) notifyLeaderDisableTopicWriteFast(topicInfo coordLog.Infof("failed to get rpc client: %v, %v", err, topicInfo.Leader) return err } - err = c.DisableTopicWriteFast(nlcoord.leaderNode.Epoch, topicInfo) + err = c.DisableTopicWriteFast(nlcoord.GetLookupLeader().Epoch, topicInfo) return err } @@ -309,7 +309,7 @@ func (nlcoord *NsqLookupCoordinator) notifyLeaderDisableTopicWrite(topicInfo *To coordLog.Infof("failed to get rpc client: %v, %v", err, topicInfo.Leader) return err } - err = c.DisableTopicWrite(nlcoord.leaderNode.Epoch, topicInfo) + err = c.DisableTopicWrite(nlcoord.GetLookupLeader().Epoch, topicInfo) return err } @@ -322,7 +322,7 @@ func (nlcoord *NsqLookupCoordinator) notifyISRDisableTopicWrite(topicInfo *Topic if err != nil { return node, err } - err = c.DisableTopicWrite(nlcoord.leaderNode.Epoch, topicInfo) + err = c.DisableTopicWrite(nlcoord.GetLookupLeader().Epoch, topicInfo) if err != nil { return node, err } @@ -339,7 +339,7 @@ func (nlcoord *NsqLookupCoordinator) notifyEnableTopicWrite(topicInfo *TopicPart if err != nil { return err } - err = c.EnableTopicWrite(nlcoord.leaderNode.Epoch, topicInfo) + err = c.EnableTopicWrite(nlcoord.GetLookupLeader().Epoch, topicInfo) if err != nil { return err } @@ -348,7 +348,7 @@ func (nlcoord *NsqLookupCoordinator) notifyEnableTopicWrite(topicInfo *TopicPart if err != nil { return err } - err = c.EnableTopicWrite(nlcoord.leaderNode.Epoch, topicInfo) + err = c.EnableTopicWrite(nlcoord.GetLookupLeader().Epoch, topicInfo) return err } diff --git a/consistence/nsqlookup_coordinator.go b/consistence/nsqlookup_coordinator.go index c39d2b2d..49039d4b 100644 --- a/consistence/nsqlookup_coordinator.go +++ b/consistence/nsqlookup_coordinator.go @@ -2,6 +2,7 @@ package consistence import ( "errors" + "fmt" "math/rand" "net" "runtime" @@ -54,6 +55,11 @@ type JoinISRState struct { isLeadershipWait bool } +func (s *JoinISRState) String() string { + return fmt.Sprintf("join %v, session %v, start %s, readys %v, isLeader %v", + s.waitingJoin, s.waitingSession, s.waitingStart, s.readyNodes, s.isLeadershipWait) +} + type RpcFailedInfo struct { nodeID string topic string @@ -89,6 +95,7 @@ type Options struct { type NsqLookupCoordinator struct { clusterKey string myNode NsqLookupdNodeInfo + leaderMu sync.RWMutex leaderNode NsqLookupdNodeInfo leadership NSQLookupdLeadership nodesMutex sync.RWMutex @@ -238,11 +245,13 @@ func (nlcoord *NsqLookupCoordinator) handleLeadership() { coordLog.Warningln("leader is lost.") continue } - if l.GetID() != nlcoord.leaderNode.GetID() || - l.Epoch != nlcoord.leaderNode.Epoch { - coordLog.Infof("lookup leader changed from %v to %v", nlcoord.leaderNode, *l) + if l.GetID() != nlcoord.GetLookupLeader().GetID() || + l.Epoch != nlcoord.GetLookupLeader().Epoch { + coordLog.Infof("lookup leader changed from %v to %v", nlcoord.GetLookupLeader(), *l) + nlcoord.leaderMu.Lock() nlcoord.leaderNode = *l - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { + nlcoord.leaderMu.Unlock() + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { // remove watchers. if nlcoord.nsqdMonitorChan != nil { close(nlcoord.nsqdMonitorChan) @@ -251,7 +260,7 @@ func (nlcoord *NsqLookupCoordinator) handleLeadership() { } nlcoord.notifyLeaderChanged(nlcoord.nsqdMonitorChan) } - if nlcoord.leaderNode.GetID() == "" { + if nlcoord.GetLookupLeader().GetID() == "" { coordLog.Warningln("leader is missing.") } case <-ticker.C: @@ -265,8 +274,8 @@ func (nlcoord *NsqLookupCoordinator) handleLeadership() { } func (nlcoord *NsqLookupCoordinator) notifyLeaderChanged(monitorChan chan struct{}) { - if nlcoord.leaderNode.GetID() != nlcoord.myNode.GetID() { - coordLog.Infof("I am slave (%v). Leader is: %v", nlcoord.myNode, nlcoord.leaderNode) + if nlcoord.GetLookupLeader().GetID() != nlcoord.myNode.GetID() { + coordLog.Infof("I am slave (%v). Leader is: %v", nlcoord.myNode, nlcoord.GetLookupLeader()) nlcoord.nodesMutex.Lock() nlcoord.removingNodes = make(map[string]string) nlcoord.nodesMutex.Unlock() @@ -1457,7 +1466,7 @@ func (nlcoord *NsqLookupCoordinator) initJoinStateAndWait(topicInfo *TopicPartit state.readyNodes = make(map[string]struct{}) state.readyNodes[topicInfo.Leader] = struct{}{} - coordLog.Infof("topic %v isr waiting session init : %v", topicInfo.GetTopicDesp(), state) + coordLog.Infof("topic %v isr waiting session init : %s", topicInfo.GetTopicDesp(), state.String()) if len(topicInfo.ISR) <= 1 { rpcErr := nlcoord.notifyISRTopicMetaInfo(topicInfo) state.waitingJoin = false @@ -1492,7 +1501,7 @@ func (nlcoord *NsqLookupCoordinator) initJoinStateAndWait(topicInfo *TopicPartit func (nlcoord *NsqLookupCoordinator) notifySingleNsqdForTopicReload(topicInfo TopicPartitionMetaInfo, nodeID string) *CoordErr { // TODO: maybe should disable write if reload node is in isr. - rpcErr := nlcoord.sendTopicInfoToNsqd(nlcoord.leaderNode.Epoch, nodeID, &topicInfo) + rpcErr := nlcoord.sendTopicInfoToNsqd(nlcoord.GetLookupLeader().Epoch, nodeID, &topicInfo) if rpcErr != nil { coordLog.Infof("failed to notify topic %v info to %v : %v", topicInfo.GetTopicDesp(), nodeID, rpcErr) if rpcErr.IsEqual(ErrTopicCoordStateInvalid) { @@ -1505,7 +1514,7 @@ func (nlcoord *NsqLookupCoordinator) notifySingleNsqdForTopicReload(topicInfo To coordLog.Infof("get leader session failed: %v", err) return &CoordErr{err.Error(), RpcCommonErr, CoordNetErr} } - return nlcoord.sendTopicLeaderSessionToNsqd(nlcoord.leaderNode.Epoch, nodeID, &topicInfo, leaderSession, "") + return nlcoord.sendTopicLeaderSessionToNsqd(nlcoord.GetLookupLeader().Epoch, nodeID, &topicInfo, leaderSession, "") } func (nlcoord *NsqLookupCoordinator) notifyAllNsqdsForTopicReload(topicInfo TopicPartitionMetaInfo) *CoordErr { @@ -1714,7 +1723,7 @@ func (nlcoord *NsqLookupCoordinator) handleReadyForISR(topic string, partition i defer state.Unlock() if !state.waitingJoin || state.waitingSession != joinISRSession { go nlcoord.triggerCheckTopicsRandom(topicInfo.Name, topicInfo.Partition, time.Second) - coordLog.Infof("%v state mismatch: %v, request join session: %v", topicInfo.GetTopicDesp(), state, joinISRSession) + coordLog.Infof("%v state mismatch: %s, request join session: %v", topicInfo.GetTopicDesp(), state, joinISRSession) return } @@ -1737,7 +1746,7 @@ func (nlcoord *NsqLookupCoordinator) handleReadyForISR(topic string, partition i coordLog.Infof("the isr nodes: %v not consistence", wrongISR) return } - coordLog.Infof("topic %v isr new state is ready for all: %v", topicInfo.GetTopicDesp(), state) + coordLog.Infof("topic %v isr new state is ready for all: %s", topicInfo.GetTopicDesp(), state) if len(topicInfo.ISR) > topicInfo.Replica/2 { rpcErr = nlcoord.notifyEnableTopicWrite(topicInfo) if rpcErr != nil { @@ -1933,13 +1942,13 @@ func (nlcoord *NsqLookupCoordinator) handleRequestNewTopicInfo(topic string, par coordLog.Infof("get topic info failed : %v", err.Error()) return nil } - nlcoord.sendTopicInfoToNsqd(nlcoord.leaderNode.Epoch, nodeID, topicInfo) + nlcoord.sendTopicInfoToNsqd(nlcoord.GetLookupLeader().Epoch, nodeID, topicInfo) leaderSession, err := nlcoord.leadership.GetTopicLeaderSession(topicInfo.Name, topicInfo.Partition) if err != nil { coordLog.Infof("get leader session failed: %v", err) return nil } - nlcoord.sendTopicLeaderSessionToNsqd(nlcoord.leaderNode.Epoch, nodeID, topicInfo, leaderSession, "") + nlcoord.sendTopicLeaderSessionToNsqd(nlcoord.GetLookupLeader().Epoch, nodeID, topicInfo, leaderSession, "") return nil } diff --git a/consistence/nsqlookup_coordinator_test.go b/consistence/nsqlookup_coordinator_test.go index a6a428b2..bcddec8c 100644 --- a/consistence/nsqlookup_coordinator_test.go +++ b/consistence/nsqlookup_coordinator_test.go @@ -49,8 +49,6 @@ func adjustLogger(t *testing.T) { SetCoordLogger(levellogger.NewSimpleLog(), levellogger.LOG_INFO) glog.SetFlags(0, "", "", true, true, 1) glog.StartWorker(time.Second) - } else { - SetCoordLogger(newTestLogger(t), levellogger.LOG_WARN) } } @@ -105,15 +103,21 @@ func (self *FakeNsqlookupLeadership) GetTopicsMetaInfoMap(topics []string) (map[ } func (self *FakeNsqlookupLeadership) GetClusterEpoch() (EpochType, error) { + self.dataMutex.Lock() + defer self.dataMutex.Unlock() return self.clusterEpoch, nil } func (self *FakeNsqlookupLeadership) Register(value *NsqLookupdNodeInfo) error { + self.dataMutex.Lock() + defer self.dataMutex.Unlock() self.fakeLeader = value return nil } func (self *FakeNsqlookupLeadership) Unregister(v *NsqLookupdNodeInfo) error { + self.dataMutex.Lock() + defer self.dataMutex.Unlock() self.fakeLeader = nil coordLog.Infof("unregistered nsqlookup: %v", v) return nil @@ -124,7 +128,9 @@ func (self *FakeNsqlookupLeadership) Stop() { } func (self *FakeNsqlookupLeadership) changeLookupLeader(newLeader *NsqLookupdNodeInfo) { + self.dataMutex.Lock() self.fakeLeader = newLeader + self.dataMutex.Unlock() select { case self.leaderChanged <- struct{}{}: case <-self.exitChan: @@ -134,6 +140,8 @@ func (self *FakeNsqlookupLeadership) changeLookupLeader(newLeader *NsqLookupdNod func (self *FakeNsqlookupLeadership) GetAllLookupdNodes() ([]NsqLookupdNodeInfo, error) { v := make([]NsqLookupdNodeInfo, 0) + self.dataMutex.Lock() + defer self.dataMutex.Unlock() v = append(v, *self.fakeLeader) return v, nil } @@ -145,8 +153,14 @@ func (self *FakeNsqlookupLeadership) AcquireAndWatchLeader(leader chan *NsqLooku case <-stopChan: return case <-self.leaderChanged: + var ld NsqLookupdNodeInfo + self.dataMutex.Lock() + if self.fakeLeader != nil { + ld = *self.fakeLeader + } + self.dataMutex.Unlock() select { - case leader <- self.fakeLeader: + case leader <- &ld: case <-self.exitChan: return } @@ -166,6 +180,7 @@ func (self *FakeNsqlookupLeadership) UpdateLookupEpoch(oldGen EpochType) (EpochT func (self *FakeNsqlookupLeadership) addFakedNsqdNode(n NsqdNodeInfo) { self.dataMutex.Lock() self.fakeNsqdNodes[n.GetID()] = n + self.clusterEpoch++ self.dataMutex.Unlock() coordLog.Infof("add fake node: %v", n) select { @@ -173,19 +188,18 @@ func (self *FakeNsqlookupLeadership) addFakedNsqdNode(n NsqdNodeInfo) { default: } - self.clusterEpoch++ } func (self *FakeNsqlookupLeadership) removeFakedNsqdNode(nid string) { self.dataMutex.Lock() delete(self.fakeNsqdNodes, nid) + self.clusterEpoch++ self.dataMutex.Unlock() coordLog.Infof("remove fake node: %v", nid) select { case self.nodeChanged <- struct{}{}: default: } - self.clusterEpoch++ } func (self *FakeNsqlookupLeadership) GetNsqdNodes() ([]NsqdNodeInfo, error) { @@ -520,6 +534,8 @@ func (self *FakeNsqlookupLeadership) AcquireTopicLeader(topic string, partition leaderSession.Session = "fake-leader-session-" + nodeData.GetID() self.updateTopicLeaderSession(topic, partition, leaderSession) coordLog.Infof("leader session update to : %v", leaderSession) + self.dataMutex.Lock() + defer self.dataMutex.Unlock() self.clusterEpoch++ return nil } @@ -541,6 +557,8 @@ func (self *FakeNsqlookupLeadership) ReleaseTopicLeader(topic string, partition case self.leaderSessionChanged <- l: default: } + self.dataMutex.Lock() + defer self.dataMutex.Unlock() self.clusterEpoch++ return nil } @@ -659,7 +677,7 @@ func waitClusterStable(lookupd *NsqLookupCoordinator, waitTime time.Duration) bo } func TestNsqLookupLeadershipChange(t *testing.T) { - SetCoordLogger(newTestLogger(t), levellogger.LOG_DEBUG) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_DEBUG) coord1, _, node1 := startNsqLookupCoord(t, true) coord2, _, node2 := startNsqLookupCoord(t, true) fakeLeadership1 := coord1.leadership.(*FakeNsqlookupLeadership) @@ -708,7 +726,7 @@ func testNsqLookupNsqdNodesChange(t *testing.T, useFakeLeadership bool) { lookupCoord1.DeleteTopic(topic3, "**") time.Sleep(time.Second) defer func() { - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) lookupCoord1.DeleteTopic(topic, "**") lookupCoord1.DeleteTopic(topic3, "**") time.Sleep(time.Second * 3) @@ -1082,7 +1100,7 @@ func TestNsqLookupNsqdCreateTopicWithChannelAutoCreateDisableDowngrade(t *testin } } - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupNsqdCreateTopic(t *testing.T) { @@ -1248,7 +1266,7 @@ func TestNsqLookupNsqdCreateTopic(t *testing.T) { newMeta, _, err := lookupCoord1.leadership.GetTopicMetaInfo(topic_p2_r2) test.Nil(t, err) test.Equal(t, oldMeta, newMeta) - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupNsqdCreateTopicFailPartition(t *testing.T) { @@ -1453,7 +1471,7 @@ func TestNsqLookupUpdateTopicMeta(t *testing.T) { test.Equal(t, int32(3), dinfo.RetentionDay) } } - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupUpdateTopicRegisterChannel(t *testing.T) { @@ -1515,7 +1533,7 @@ func TestNsqLookupUpdateTopicRegisterChannel(t *testing.T) { } } - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupMarkNodeRemove(t *testing.T) { @@ -1624,7 +1642,7 @@ func TestNsqLookupMarkNodeRemove(t *testing.T) { if time.Since(checkStart) >= time.Minute*2 { t.Error("remove node timeout") } - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupExpandPartition(t *testing.T) { @@ -1730,7 +1748,7 @@ func TestNsqLookupExpandPartition(t *testing.T) { // should fail err = lookupCoord.ExpandTopicPartition(topic_p1_r3, 3) test.NotNil(t, err) - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupMovePartition(t *testing.T) { @@ -2005,7 +2023,7 @@ func TestNsqLookupMovePartition(t *testing.T) { test.Equal(t, FindSlice(t0.ISR, toNode) != -1, true) test.Equal(t, -1, FindSlice(t0.ISR, fromNode)) - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupMovePartitionRetryWhileLocalRemoved(t *testing.T) { @@ -2170,7 +2188,7 @@ func TestNsqLookupMovePartitionRetryWhileLocalRemoved(t *testing.T) { test.Equal(t, FindSlice(t0.ISR, toNode) != -1, true) test.Equal(t, -1, FindSlice(t0.ISR, fromNode)) - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func getTopicLeaderNode(t *testing.T, lookupLeadership NSQLookupdLeadership, topic string, pid int, nodeInfoList map[string]*testClusterNodeInfo) *testClusterNodeInfo { @@ -2350,7 +2368,7 @@ func TestNsqLookupSlaveTimeoutReadUncommitted(t *testing.T) { } } - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupMovePartitionAndSlaveTimeoutWhileReadWrite(t *testing.T) { @@ -2403,7 +2421,7 @@ func TestNsqLookupMovePartitionAndSlaveTimeoutWhileReadWrite(t *testing.T) { defer wg.Done() wcnt := 0 for { - time.Sleep(time.Millisecond * 10) + time.Sleep(time.Millisecond * 50) select { case <-stopC: t.Logf("write %v cnt at end: %v", wcnt, time.Now()) @@ -2426,6 +2444,7 @@ func TestNsqLookupMovePartitionAndSlaveTimeoutWhileReadWrite(t *testing.T) { if err == nil { wcnt++ atomic.AddInt32(&totalPub, 1) + localT.ForceFlushForChannels(true) } } } @@ -2467,7 +2486,7 @@ func TestNsqLookupMovePartitionAndSlaveTimeoutWhileReadWrite(t *testing.T) { if ch.IsConfirmed(msg) { continue } - ch.StartInFlightTimeout(msg, NewFakeConsumer(1), "", time.Second) + ch.StartInFlightTimeout(msg, NewFakeConsumer(1), "", time.Second*3) if stoppedTime == nil { time.Sleep(time.Millisecond * time.Duration(rand.Intn(10))) } @@ -2500,8 +2519,9 @@ func TestNsqLookupMovePartitionAndSlaveTimeoutWhileReadWrite(t *testing.T) { stoppedTime = &tn } if ch.Depth() <= 0 && stoppedTime != nil { + localT.ForceFlush() if time.Since(*stoppedTime) > time.Second*5 { - t.Logf("consumed %v cnt, depth: %v at end: %v", cnt, ch.Depth(), time.Now()) + t.Logf("consumed %v cnt, depth: %v,%v at end: %v", cnt, ch.Depth(), ch.GetDelayedQueueCnt(), time.Now()) return } } @@ -2541,18 +2561,18 @@ func TestNsqLookupMovePartitionAndSlaveTimeoutWhileReadWrite(t *testing.T) { setTestSlaveTimeout(true) select { case <-stopC: - case <-time.After(time.Second * MAX_WRITE_RETRY * 3): + case <-time.After(time.Second * MAX_WRITE_RETRY * 2): } setTestSlaveTimeout(false) select { case <-stopC: return - case <-time.After(time.Second * MAX_WRITE_RETRY * 10): + case <-time.After(time.Second * MAX_WRITE_RETRY * 5): } } }() for { - if time.Since(start) > time.Minute*5 { + if time.Since(start) > time.Minute*2 { break } // TODO: reset queue end to make sure we can consume all @@ -2700,7 +2720,7 @@ func TestNsqLookupMovePartitionAndSlaveTimeoutWhileReadWrite(t *testing.T) { break } } - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) time.Sleep(time.Second) } @@ -2777,7 +2797,7 @@ func testNsqLookupOrderedOrMultiPartTopicCreate(t *testing.T, tnameSuffix string newMeta, _, err := lookupCoord1.leadership.GetTopicMetaInfo(topic_p25_r3) test.Nil(t, err) test.Equal(t, oldMeta, newMeta) - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func checkOrderedMultiTopic(t *testing.T, topic string, expectedPart int, aliveNode int, @@ -2984,7 +3004,7 @@ func testNsqLookupOrderedOrMultiPartTopicBalance(t *testing.T, tname string, ord checkOrderedMultiTopic(t, topic_p13_r2, 13, len(nodeInfoList), nodeInfoList, lookupLeadership, true) - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupTopNTopicBalance(t *testing.T) { @@ -3184,7 +3204,7 @@ func TestNsqLookupTopNTopicBalance(t *testing.T) { assert.True(t, maxLeaderNum-minLeaderNum <= topNBalanceDiff+3) assert.True(t, maxAllNum-minAllNum <= topNBalanceDiff+3) - SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) + //SetCoordLogger(newTestLogger(t), levellogger.LOG_ERR) } func TestNsqLookupConsumeTopicWhileNodeStop(t *testing.T) { diff --git a/consistence/topic_coordinator.go b/consistence/topic_coordinator.go index 284ee105..de35f698 100644 --- a/consistence/topic_coordinator.go +++ b/consistence/topic_coordinator.go @@ -124,7 +124,6 @@ type coordData struct { syncedConsumeMgr *ChannelConsumeMgr logMgr *TopicCommitLogMgr delayedLogMgr *TopicCommitLogMgr - forceLeave int32 } func (cd *coordData) updateBufferSize(bs int) { @@ -161,6 +160,7 @@ func (cd *coordData) GetCopy() *coordData { type TopicCoordinator struct { dataMutex sync.Mutex *coordData + forceLeave int32 // hold for write to avoid disable or exiting or catchup // lock order: first lock writehold then lock data to avoid deadlock writeHold sync.Mutex @@ -300,6 +300,32 @@ func (tc *TopicCoordinator) Exiting() { atomic.StoreInt32(&tc.exiting, 1) } +func (tc *TopicCoordinator) checkWriteForLeader(myID string) *CoordErr { + if tc.IsForceLeave() { + return ErrNotTopicLeader + } + cd := tc.GetData() + if cd.GetLeaderSessionID() != myID || cd.topicInfo.Leader != myID { + return ErrNotTopicLeader + } + if cd.topicLeaderSession.Session == "" { + return ErrMissingTopicLeaderSession + } + return nil +} + +func (tc *TopicCoordinator) SetForceLeave(leave bool) { + if leave { + atomic.StoreInt32(&tc.forceLeave, 1) + } else { + atomic.StoreInt32(&tc.forceLeave, 0) + } +} + +func (tc *TopicCoordinator) IsForceLeave() bool { + return atomic.LoadInt32(&tc.forceLeave) == 1 +} + func (cd *coordData) GetLeader() string { return cd.topicInfo.Leader } @@ -336,35 +362,6 @@ func (cd *coordData) GetTopicEpochForWrite() EpochType { return cd.topicInfo.EpochForWrite } -func (cd *TopicCoordinator) checkWriteForLeader(myID string) *CoordErr { - return cd.GetData().checkWriteForLeader(myID) -} - -func (cd *coordData) checkWriteForLeader(myID string) *CoordErr { - if cd.IsForceLeave() { - return ErrNotTopicLeader - } - if cd.GetLeaderSessionID() != myID || cd.topicInfo.Leader != myID { - return ErrNotTopicLeader - } - if cd.topicLeaderSession.Session == "" { - return ErrMissingTopicLeaderSession - } - return nil -} - func (cd *coordData) IsISRReadyForWrite(myID string) bool { return (len(cd.topicInfo.ISR) > cd.topicInfo.Replica/2) && cd.IsMineISR(myID) } - -func (cd *coordData) SetForceLeave(leave bool) { - if leave { - atomic.StoreInt32(&cd.forceLeave, 1) - } else { - atomic.StoreInt32(&cd.forceLeave, 0) - } -} - -func (cd *coordData) IsForceLeave() bool { - return atomic.LoadInt32(&cd.forceLeave) == 1 -} diff --git a/go.mod b/go.mod index 0b4ec95d..4240759d 100644 --- a/go.mod +++ b/go.mod @@ -4,8 +4,9 @@ go 1.13 require ( github.com/BurntSushi/toml v0.3.1 + github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46 // indirect github.com/Workiva/go-datastructures v1.0.50 - github.com/absolute8511/bolt v1.5.0 + github.com/absolute8511/bolt v1.5.1 github.com/absolute8511/glog v0.3.1 github.com/absolute8511/gorpc v0.0.0-20161203145636-60ee7d4359cb github.com/absolute8511/goskiplist v0.0.0-20170727031420-3ba6f667c3df @@ -16,11 +17,15 @@ require ( github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 // indirect github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b github.com/cenkalti/backoff v2.1.0+incompatible - github.com/coreos/etcd v2.3.8+incompatible + github.com/certifi/gocertifi v0.0.0-20200211180108-c7c1fbc02894 // indirect + github.com/cockroachdb/pebble v0.0.0-20210322142411-65860c8c27ac + github.com/coreos/etcd v3.3.13+incompatible + github.com/getsentry/raven-go v0.2.0 // indirect + github.com/go-ole/go-ole v1.2.5 // indirect github.com/gobwas/glob v0.2.3 - github.com/gogo/protobuf v1.2.1 + github.com/gogo/protobuf v1.3.1 github.com/golang/protobuf v1.4.2 - github.com/golang/snappy v0.0.1 + github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf github.com/gorilla/sessions v1.1.3 github.com/hashicorp/golang-lru v0.5.3 github.com/json-iterator/go v1.1.10 @@ -30,18 +35,22 @@ require ( github.com/mreiferson/go-options v0.0.0-20161229190002-77551d20752b github.com/myesui/uuid v1.0.0 // indirect github.com/prometheus/client_golang v1.7.1 + github.com/shirou/gopsutil v3.21.2+incompatible github.com/spaolacci/murmur3 v1.1.0 - github.com/stretchr/testify v1.4.0 + github.com/stretchr/testify v1.6.1 github.com/tidwall/gjson v1.1.3 github.com/tidwall/match v1.0.1 // indirect github.com/twinj/uuid v1.0.0 + github.com/twmb/murmur3 v1.1.5 github.com/valyala/fastjson v1.6.1 github.com/viki-org/dnscache v0.0.0-20130720023526-c70c1f23c5d8 github.com/youzan/go-nsq v1.7.2-HA - golang.org/x/net v0.0.0-20190613194153-d28f0bde5980 - golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e + golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7 + golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e - google.golang.org/grpc v1.26.0 + google.golang.org/grpc v1.29.1 gopkg.in/stretchr/testify.v1 v1.2.2 // indirect - gopkg.in/yaml.v2 v2.2.5 + gopkg.in/yaml.v2 v2.3.0 ) + +replace github.com/ugorji/go => github.com/ugorji/go/codec v1.1.7 diff --git a/go.sum b/go.sum index 6d50f418..de3c98b8 100644 --- a/go.sum +++ b/go.sum @@ -1,20 +1,37 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= +github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= +github.com/DataDog/zstd v1.4.5 h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ= +github.com/DataDog/zstd v1.4.5/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= +github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= +github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= +github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= +github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46 h1:5sXbqlSomvdjlRbWyNqkPsJ3Fg+tQZCbgeX1VGljbQY= +github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/Workiva/go-datastructures v1.0.50 h1:slDmfW6KCHcC7U+LP3DDBbm4fqTwZGn1beOFPfGaLvo= github.com/Workiva/go-datastructures v1.0.50/go.mod h1:Z+F2Rca0qCsVYDS8z7bAGm8f3UkzuWYS/oBZz5a7VVA= github.com/absolute8511/bolt v1.5.0 h1:1HMP0GLSCHFStWt5VwwqS0pb1msmP+BsKmJ3TmZd21Y= github.com/absolute8511/bolt v1.5.0/go.mod h1:ZvG4+BG8/gnqo3YxXstB96fsOPZfm7hVg9xnud/um0A= +github.com/absolute8511/bolt v1.5.1 h1:DhzQahV6eJUjLGtk8S72CNVAmXVq8XIs8AVQINY47IY= +github.com/absolute8511/bolt v1.5.1/go.mod h1:ZcWITTZhq+Amag5xYMjaRvTLA1eeN4AtQ4pXsdDQh7g= github.com/absolute8511/glog v0.3.1 h1:oTuACTGSouUxi7CWuya5azGQ9s7WnsAxrAxAZPKevRk= github.com/absolute8511/glog v0.3.1/go.mod h1:T44AH/EtS/Grr3Ujq6U0Lhk1y4960FC9i06pxXblZ2I= github.com/absolute8511/gorpc v0.0.0-20161203145636-60ee7d4359cb h1:F8aNeq5n+KBvboy1eAlpCcZ+DqdJ4a3g+CQ7L0jwt4o= github.com/absolute8511/gorpc v0.0.0-20161203145636-60ee7d4359cb/go.mod h1:PjNRfcxNGdDHKsjOwvm3QSXPsL1PCSCPkI0qqIbL9Fs= github.com/absolute8511/goskiplist v0.0.0-20170727031420-3ba6f667c3df h1:7iX7qyzKpDQ5ymyZrrlhK99T9phGo44DBhg9muteGXs= github.com/absolute8511/goskiplist v0.0.0-20170727031420-3ba6f667c3df/go.mod h1:2lOX2xSM21N/twUqgjVkZPtyHgOCex2dMxNZehggL/8= +github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= +github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= @@ -34,28 +51,87 @@ github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D github.com/cenkalti/backoff v2.1.0+incompatible h1:FIRvWBZrzS4YC7NT5cOuZjexzFvIr+Dbi6aD1cZaNBk= github.com/cenkalti/backoff v2.1.0+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/certifi/gocertifi v0.0.0-20200211180108-c7c1fbc02894 h1:JLaf/iINcLyjwbtTsCJjc6rtlASgHeIJPrB6QmwURnA= +github.com/certifi/gocertifi v0.0.0-20200211180108-c7c1fbc02894/go.mod h1:sGbDF6GwGcLpkNXPUTkMRoywsNa/ol15pxFe6ERfguA= github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= +github.com/cockroachdb/errors v1.2.4 h1:Lap807SXTH5tri2TivECb/4abUkMZC9zRoLarvcKDqs= +github.com/cockroachdb/errors v1.2.4/go.mod h1:rQD95gz6FARkaKkQXUksEje/d9a6wBJoCr5oaCLELYA= +github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= +github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQu+Y= +github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= +github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= +github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= +github.com/cockroachdb/pebble v0.0.0-20200616214509-8de6baeca713 h1:SK0BUmdsXF7Ammqw+AmaHOlzFgkmtyDM/WRCTwDpS6I= +github.com/cockroachdb/pebble v0.0.0-20200616214509-8de6baeca713/go.mod h1:crLnbSFbwAcQNs9FPfI1avHb5BqVgqZcr4r+IzpJ5FM= +github.com/cockroachdb/pebble v0.0.0-20210322142411-65860c8c27ac h1:jdmG9rMNSrdqji2lTUrehJH/mTvhSZeJIplS5Ehr/8E= +github.com/cockroachdb/pebble v0.0.0-20210322142411-65860c8c27ac/go.mod h1:1XpB4cLQcF189RAcWi4gUc110zJgtOfT7SVNGY8sOe0= +github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= +github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= +github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= +github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= github.com/coreos/etcd v2.3.8+incompatible h1:Lkp5dgqMANTjq0UW74OP1H8yCDQT0In4jrw6xfcNlGE= github.com/coreos/etcd v2.3.8+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/etcd v3.3.13+incompatible h1:8F3hqu9fGYLBifCmRCJsicFqDx/D68Rt3q1JMazcgBQ= +github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= +github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY= +github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= +github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= +github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= +github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= +github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= +github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= github.com/frankban/quicktest v1.7.2 h1:2QxQoC1TS09S7fhCPsrvqYdvP1H5M1P1ih5ABm3BTYk= github.com/frankban/quicktest v1.7.2/go.mod h1:jaStnuzAqU1AJdCO0l53JDCJrVDKcS03DbaAcR7Ks/o= +github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= +github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= +github.com/getsentry/raven-go v0.2.0 h1:no+xWJRb5ZI7eE8TWgIq1jLulQiIoLG0IfYxv5JYMGs= +github.com/getsentry/raven-go v0.2.0/go.mod h1:KungGk8q33+aIAZUIVWZDr2OfAEBsO49PX4NzFV5kcQ= +github.com/ghemawat/stream v0.0.0-20171120220530-696b145b53b9/go.mod h1:106OIgooyS7OzLDOpUGgm9fA3bQENb/cFSyyBmMoJDs= +github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= +github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= +github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= +github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= +github.com/go-ole/go-ole v1.2.5 h1:t4MGB5xEDZvXI+0rMjjsfBsD7yAgp/s9ZDkL1JndXwY= +github.com/go-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gobwas/glob v0.2.3 h1:A4xDbljILXROh+kObIiy5kIaPYD8e96x1tgBhUI5J+Y= github.com/gobwas/glob v0.2.3/go.mod h1:d3Ez4x06l9bZtSvzIay5+Yzi0fmZzPgnTbPcKjJAkT8= +github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= +github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= +github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= +github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE= -github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= +github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.3.1 h1:DqDEcV5aeaTmdFBePNpYsp3FlcVH/2ISVVM9Qf8PSls= +github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -64,6 +140,7 @@ github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= @@ -73,6 +150,9 @@ github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0 github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf h1:gFVkHXmVAhEbxZVDln5V9GKrLaluNoFHDbrZwAWZgws= +github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/go-cmp v0.2.0 h1:+dTQ8DZQJz0Mb/HjFlkptS1FeQ4cWSnN941F8aEG4SQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -80,24 +160,51 @@ github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/context v1.1.1 h1:AWwleXJkX/nhcU9bZSnZoi3h/qGYqQAGhq6zZe/aQW8= github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= github.com/gorilla/securecookie v1.1.1 h1:miw7JPhV+b/lAHSXz4qd/nN9jRiAFV5FwjeKyCS8BvQ= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.1.3 h1:uXoZdcdA5XdXF3QzuSlheVRUvjl+1rKY7zBXL68L9RU= github.com/gorilla/sessions v1.1.3/go.mod h1:8KCfur6+4Mqcc6S0FEfKuN15Vl5MgXW92AE8ovaJD0w= +github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= +github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= github.com/hashicorp/golang-lru v0.5.3 h1:YPkqC67at8FYaadspW/6uE0COsBxS2656RLEr8Bppgk= github.com/hashicorp/golang-lru v0.5.3/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= +github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= +github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= +github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= +github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= +github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= +github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= +github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.10 h1:Kz6Cvnvv2wGdaG/V8yMvfkmNiXq9Ya2KUv4rouJJr68= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/judwhite/go-svc v1.0.0 h1:W447kYhZsqC14hkfNG8XLy9wbYibeMW75g5DtAIpFGw= github.com/judwhite/go-svc v1.0.0/go.mod h1:EeMSAFO3mLgEQfcvnZ50JDG0O1uQlagpAbMS6talrXE= +github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= +github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= +github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= github.com/julienschmidt/httprouter v1.2.0 h1:TDTW5Yz1mjftljbcKqRcrYhd4XeOoI98t+9HbQbYf7g= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= +github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= +github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= +github.com/kataras/iris/v12 v12.0.1/go.mod h1:udK4vLQKkdDqMGJJVd/msuMtN6hpYJhg/lSzuxjhO+U= +github.com/kataras/neffos v0.0.10/go.mod h1:ZYmJC07hQPW67eKuzlfY7SO3bC0mw83A3j6im82hfqw= +github.com/kataras/pio v0.0.0-20190103105442-ea782b38602d/go.mod h1:NV88laa9UiiDuX9AhMbDPkGYSPugBOV6yTZB1l2K9Z0= +github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.11.7 h1:0hzRabrMN4tSTvMfnL3SCv1ZGeAP23ynzodBgaHeMeg= +github.com/klauspost/compress v1.11.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -106,23 +213,50 @@ github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfn github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= +github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= +github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= +github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= +github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= +github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= +github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQGvsUt6O3Pj+LDCQ= +github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mreiferson/go-options v0.0.0-20161229190002-77551d20752b h1:xjKomx939vefURtocD1uaKvcvAp1dNYX05i0TIpnfVI= github.com/mreiferson/go-options v0.0.0-20161229190002-77551d20752b/go.mod h1:A0JOgZNsj9V+npbgxH0Ib75PvrHS6Ezri/4HdcTp/DI= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/myesui/uuid v1.0.0 h1:xCBmH4l5KuvLYc5L7AS7SZg9/jKdIFubM7OVoLqaQUI= github.com/myesui/uuid v1.0.0/go.mod h1:2CDfNgU0LR8mIdO8vdWd8i9gWWxLlcoIGGpSNgafq84= +github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= +github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= +github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= +github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= +github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= +github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= +github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pierrec/lz4 v2.4.1+incompatible h1:mFe7ttWaflA46Mhqh+jUfjp2qTbPYxLB2/OyBppH9dg= github.com/pierrec/lz4 v2.4.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= +github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= @@ -141,80 +275,169 @@ github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.1.3 h1:F0+tqvhOksq22sc6iCHF5WGlWjdwj92p0udFh1VFBS8= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= +github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= +github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= +github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= +github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= +github.com/shirou/gopsutil v3.21.2+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= +github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= +github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= +github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= +github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= +github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/tidwall/gjson v1.1.3 h1:u4mspaByxY+Qk4U1QYYVzGFI8qxN/3jtEV0ZDb2vRic= github.com/tidwall/gjson v1.1.3/go.mod h1:c/nTNbUr0E0OrXEhq1pwa8iEgc2DOt4ZZqAt1HtCkPA= github.com/tidwall/match v1.0.1 h1:PnKP62LPNxHKTwvHHZZzdOAOCtsJTjo6dZLCwpKm5xc= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/twinj/uuid v1.0.0 h1:fzz7COZnDrXGTAOHGuUGYd6sG+JMq+AoE7+Jlu0przk= github.com/twinj/uuid v1.0.0/go.mod h1:mMgcE1RHFUFqe5AfiwlINXisXfDGro23fWdPUfOMjRY= +github.com/twmb/murmur3 v1.1.5 h1:i9OLS9fkuLzBXjt6dptlAEyk58fJsSTXbRg3SgVyqgk= +github.com/twmb/murmur3 v1.1.5/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= +github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= +github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= +github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= +github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= +github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= +github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= github.com/valyala/fastjson v1.6.1 h1:qJs/Kz/HebWzk8LmhOrSm7kdOyJBr1XB+zSkYtEEfQE= github.com/valyala/fastjson v1.6.1/go.mod h1:CLCAqky6SMuOcxStkYQvblddUtoRxhYMGLrsQns1aXY= +github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= +github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= github.com/viki-org/dnscache v0.0.0-20130720023526-c70c1f23c5d8 h1:EVObHAr8DqpoJCVv6KYTle8FEImKhtkfcZetNqxDoJQ= github.com/viki-org/dnscache v0.0.0-20130720023526-c70c1f23c5d8/go.mod h1:dniwbG03GafCjFohMDmz6Zc6oCuiqgH6tGNyXTkHzXE= +github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= +github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= +github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= +github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= +github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= github.com/youzan/go-nsq v1.7.2-HA h1:6y4vjZAIlOc6FPBwMLUL4bh8k1c0Wu0LG2dBIowgCtQ= github.com/youzan/go-nsq v1.7.2-HA/go.mod h1:ZWS/W9xoZmE6VJnHLu2AfRaBY0DdhQrpDysAHYSsGp4= +github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= +github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= +github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20200513190911-00229845015e h1:rMqLP+9XLy+LdbCXHjJHAmTfXCr93W7oruWA6Hq1Alc= +golang.org/x/exp v0.0.0-20200513190911-00229845015e/go.mod h1:4M0jN8W1tt0AVLNr8HDosyJCDCDuyL9N9+3m7wDWgKw= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a h1:gOpx8G595UYyvj8UK4+OFyY4rx037g3fmfhe5SasG3U= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980 h1:dfGZHvZk057jK2MCeWus/TowKpJ8y4AmooUzdBSR9GU= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859 h1:R/3boaszxrf1GEUWTVDzSKVwLmSJpwZ1yqXm8j0v2QI= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7 h1:AeiKBIuRw3UomYXSbLy0Mc2dDLfdtbT/IVn4keq83P0= +golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 h1:SQFwaSi55rU7vdNs9Yr0Z324VNlrF+0wMqRXT4St8ck= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527 h1:uYVVQ9WP/Ds2ROhcaGPeIdVq0RIXVLwsHlnvJ+cT1So= golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1 h1:ogLJMz+qpzav7lGMh10LMvAkM/fAoGlaiiHYiFYdm80= golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d h1:L/IKR6COd7ubZrs2oTnTi73IhgqJ71c9s80WsQnh0Es= +golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e h1:EHBhcS0mlXEAVwNyO2dLfjToGsyY4j24pTs2ScHnX7s= golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55 h1:gSJIx1SDwno+2ElGhA4+qG2zF97qiUzTM+rQ0klBOcE= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg= google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.29.1 h1:EC2SB8S04d2r73uptxphDSUG+kTKVgjRPF+N3xpxRB4= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -227,13 +450,22 @@ gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+ gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= +gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= +gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= +gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= gopkg.in/stretchr/testify.v1 v1.2.2 h1:yhQC6Uy5CqibAIlk1wlusa/MJ3iAN49/BsR/dCCKz3M= gopkg.in/stretchr/testify.v1 v1.2.2/go.mod h1:QI5V/q6UbPmuhtm10CaFZxED9NreB8PnFYN9JcR6TxU= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5 h1:ymVxjfMaHvXD8RqPRmzHHsB3VvucivSkIAvJFDI5O3c= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU= +gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/internal/clusterinfo/data.go b/internal/clusterinfo/data.go index 69c860bc..3c0bc841 100644 --- a/internal/clusterinfo/data.go +++ b/internal/clusterinfo/data.go @@ -947,11 +947,19 @@ func (c *ClusterInfo) GetClusterInfoDC(lookupdAdresses []LookupdAddressDC) ([]*C return dcClusterInfo, nil } +func (c *ClusterInfo) GetNSQDStatsWithClients(producers Producers, selectedTopic string, sortBy string, leaderOnly bool) ([]*TopicStats, map[string]*ChannelStats, error) { + return c.getNSQDStats(producers, selectedTopic, sortBy, leaderOnly, true) +} + // GetNSQDStats returns aggregate topic and channel stats from the given Producers // // if selectedTopic is empty, this will return stats for *all* topic/channels // and the ChannelStats dict will be keyed by topic + ':' + channel func (c *ClusterInfo) GetNSQDStats(producers Producers, selectedTopic string, sortBy string, leaderOnly bool) ([]*TopicStats, map[string]*ChannelStats, error) { + return c.getNSQDStats(producers, selectedTopic, sortBy, leaderOnly, false) +} + +func (c *ClusterInfo) getNSQDStats(producers Producers, selectedTopic string, sortBy string, leaderOnly bool, needClient bool) ([]*TopicStats, map[string]*ChannelStats, error) { var lock sync.Mutex var wg sync.WaitGroup var topicStatsList TopicStatsList @@ -969,7 +977,10 @@ func (c *ClusterInfo) GetNSQDStats(producers Producers, selectedTopic string, so defer wg.Done() addr := p.HTTPAddress() - endpoint := fmt.Sprintf("http://%s/stats?format=json&leaderOnly=%t&needClients=true", addr, leaderOnly) + endpoint := fmt.Sprintf("http://%s/stats?format=json&leaderOnly=%t&needClients=%t", addr, leaderOnly, needClient) + if !needClient { + endpoint = fmt.Sprintf("http://%s/stats?format=json&leaderOnly=%t", addr, leaderOnly) + } if selectedTopic != "" { endpoint = fmt.Sprintf("http://%s/stats?format=json&topic=%s&leaderOnly=%t", addr, selectedTopic, leaderOnly) } diff --git a/nsqd/channel.go b/nsqd/channel.go index ad065ba5..45a5487f 100644 --- a/nsqd/channel.go +++ b/nsqd/channel.go @@ -178,7 +178,8 @@ type Channel struct { // NewChannel creates a new instance of the Channel type and returns a pointer func NewChannel(topicName string, part int, topicOrdered bool, channelName string, chEnd BackendQueueEnd, opt *Options, deleteCallback func(*Channel), moreDataCallback func(*Channel), consumeDisabled int32, - notify INsqdNotify, ext int32, queueStart BackendQueueEnd, metaStorage IMetaStorage, forceReload bool) *Channel { + notify INsqdNotify, ext int32, queueStart BackendQueueEnd, metaStorage IMetaStorage, + kvTopic *KVTopic, forceReload bool) *Channel { c := &Channel{ topicName: topicName, @@ -254,6 +255,7 @@ func NewChannel(topicName string, part int, topicOrdered bool, channelName strin chEnd, false, metaStorage, + kvTopic, forceReload, ) @@ -2378,6 +2380,8 @@ func (c *Channel) CheckIfTimeoutToomuch(msg *Message, msgTimeout time.Duration) if toCnt > maxTimeoutCntToReq && !c.IsEphemeral() && !c.IsOrdered() { tnow := time.Now().UnixNano() if tnow-c.DepthTimestamp() > timeoutBlockingWait.Nanoseconds() { + c.inFlightMutex.Lock() + defer c.inFlightMutex.Unlock() nmsg, ok := c.checkMsgRequeueToEnd(msg, msgTimeout) if ok { if c.isTracedOrDebugTraceLog(msg) { diff --git a/nsqd/channel_test.go b/nsqd/channel_test.go index ac1c57d5..c5a32f29 100644 --- a/nsqd/channel_test.go +++ b/nsqd/channel_test.go @@ -119,13 +119,24 @@ func TestPutMessage2Chan(t *testing.T) { topic.PutMessage(msg) topic.flushBuffer(true) - outputMsg1 := <-channel1.clientMsgChan - equal(t, msg.ID, outputMsg1.ID) - equal(t, msg.Body, outputMsg1.Body) + timer := time.NewTimer(time.Second * 10) + select { + case outputMsg1 := <-channel1.clientMsgChan: + equal(t, msg.ID, outputMsg1.ID) + equal(t, msg.Body, outputMsg1.Body) + case <-timer.C: + t.Errorf("timeout waiting consume") + return + } - outputMsg2 := <-channel2.clientMsgChan - equal(t, msg.ID, outputMsg2.ID) - equal(t, msg.Body, outputMsg2.Body) + select { + case outputMsg2 := <-channel2.clientMsgChan: + equal(t, msg.ID, outputMsg2.ID) + equal(t, msg.Body, outputMsg2.Body) + case <-timer.C: + t.Errorf("timeout waiting consume") + return + } } func TestChannelBackendMaxMsgSize(t *testing.T) { @@ -818,7 +829,7 @@ func TestChannelEmptyWhileReqDelayedMessageWaitingInReq(t *testing.T) { ast.True(t, waitReqMoreCnt > 0, "should have wait more req count") ast.Equal(t, waitChCnt, realWaitChCnt+1) ast.Equal(t, 0, inflightCnt) - ast.Equal(t, int64(waitChCnt+waitReqMoreCnt), atomic.LoadInt64(&channel.deferredFromDelay)+1) + ast.InDelta(t, int64(waitChCnt+waitReqMoreCnt), atomic.LoadInt64(&channel.deferredFromDelay), 1) break } diff --git a/nsqd/codec/bytes.go b/nsqd/codec/bytes.go new file mode 100644 index 00000000..1fa9792c --- /dev/null +++ b/nsqd/codec/bytes.go @@ -0,0 +1,182 @@ +// Copyright 2015 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package codec + +import ( + "errors" + "fmt" + "runtime" + "unsafe" +) + +const ( + encGroupSize = 8 + encMarker = byte(0xFF) + encPad = byte(0x0) +) + +var ( + pads = make([]byte, encGroupSize) + encPads = []byte{encPad} +) + +// EncodeBytes guarantees the encoded value is in ascending order for comparison, +// encoding with the following rule: +// [group1][marker1]...[groupN][markerN] +// group is 8 bytes slice which is padding with 0. +// marker is `0xFF - padding 0 count` +// For example: +// [] -> [0, 0, 0, 0, 0, 0, 0, 0, 247] +// [1, 2, 3] -> [1, 2, 3, 0, 0, 0, 0, 0, 250] +// [1, 2, 3, 0] -> [1, 2, 3, 0, 0, 0, 0, 0, 251] +// [1, 2, 3, 4, 5, 6, 7, 8] -> [1, 2, 3, 4, 5, 6, 7, 8, 255, 0, 0, 0, 0, 0, 0, 0, 0, 247] +// Refer: https://github.com/facebook/mysql-5.6/wiki/MyRocks-record-format#memcomparable-format +func EncodeBytes(b []byte, data []byte) []byte { + // Allocate more space to avoid unnecessary slice growing. + // Assume that the byte slice size is about `(len(data) / encGroupSize + 1) * (encGroupSize + 1)` bytes, + // that is `(len(data) / 8 + 1) * 9` in our implement. + dLen := len(data) + reallocSize := (dLen/encGroupSize + 1) * (encGroupSize + 1) + result := reallocBytes(b, reallocSize) + for idx := 0; idx <= dLen; idx += encGroupSize { + remain := dLen - idx + padCount := 0 + if remain >= encGroupSize { + result = append(result, data[idx:idx+encGroupSize]...) + } else { + padCount = encGroupSize - remain + result = append(result, data[idx:]...) + result = append(result, pads[:padCount]...) + } + + marker := encMarker - byte(padCount) + result = append(result, marker) + } + + return result +} + +func decodeBytes(b []byte, reverse bool) ([]byte, []byte, error) { + data := make([]byte, 0, len(b)) + for { + if len(b) < encGroupSize+1 { + return nil, nil, errors.New("insufficient bytes to decode value") + } + + groupBytes := b[:encGroupSize+1] + + group := groupBytes[:encGroupSize] + marker := groupBytes[encGroupSize] + + var padCount byte + if reverse { + padCount = marker + } else { + padCount = encMarker - marker + } + if padCount > encGroupSize { + return nil, nil, fmt.Errorf("invalid marker byte, group bytes %q", groupBytes) + } + + realGroupSize := encGroupSize - padCount + data = append(data, group[:realGroupSize]...) + b = b[encGroupSize+1:] + + if padCount != 0 { + var padByte = encPad + if reverse { + padByte = encMarker + } + // Check validity of padding bytes. + for _, v := range group[realGroupSize:] { + if v != padByte { + return nil, nil, fmt.Errorf("invalid padding byte, group bytes %q", groupBytes) + } + } + break + } + } + if reverse { + reverseBytes(data) + } + return b, data, nil +} + +// DecodeBytes decodes bytes which is encoded by EncodeBytes before, +// returns the leftover bytes and decoded value if no error. +func DecodeBytes(b []byte) ([]byte, []byte, error) { + return decodeBytes(b, false) +} + +// EncodeBytesDesc first encodes bytes using EncodeBytes, then bitwise reverses +// encoded value to guarantee the encoded value is in descending order for comparison. +func EncodeBytesDesc(b []byte, data []byte) []byte { + n := len(b) + b = EncodeBytes(b, data) + reverseBytes(b[n:]) + return b +} + +// DecodeBytesDesc decodes bytes which is encoded by EncodeBytesDesc before, +// returns the leftover bytes and decoded value if no error. +func DecodeBytesDesc(b []byte) ([]byte, []byte, error) { + return decodeBytes(b, true) +} + +// See https://golang.org/src/crypto/cipher/xor.go +const wordSize = int(unsafe.Sizeof(uintptr(0))) +const supportsUnaligned = runtime.GOARCH == "386" || runtime.GOARCH == "amd64" + +func fastReverseBytes(b []byte) { + n := len(b) + w := n / wordSize + if w > 0 { + bw := *(*[]uintptr)(unsafe.Pointer(&b)) + for i := 0; i < w; i++ { + bw[i] = ^bw[i] + } + } + + for i := w * wordSize; i < n; i++ { + b[i] = ^b[i] + } +} + +func safeReverseBytes(b []byte) { + for i := range b { + b[i] = ^b[i] + } +} + +func reverseBytes(b []byte) { + if supportsUnaligned { + fastReverseBytes(b) + return + } + + safeReverseBytes(b) +} + +// reallocBytes is like realloc. +func reallocBytes(b []byte, n int) []byte { + newSize := len(b) + n + if cap(b) < newSize { + bs := make([]byte, len(b), newSize) + copy(bs, b) + return bs + } + + // slice b has capability to store n bytes + return b +} diff --git a/nsqd/codec/memcmp_codec.go b/nsqd/codec/memcmp_codec.go new file mode 100644 index 00000000..a1f48152 --- /dev/null +++ b/nsqd/codec/memcmp_codec.go @@ -0,0 +1,214 @@ +// Copyright 2015 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License.package rockredis + +package codec + +import ( + "errors" + "fmt" +) + +const ( + NilFlag byte = 0 + bytesFlag byte = 1 + intFlag byte = 3 + uintFlag byte = 4 + floatFlag byte = 5 + maxFlag byte = 250 +) + +func memcmpEncode(b []byte, vals []interface{}) ([]byte, error) { + for _, val := range vals { + switch realVal := val.(type) { + case byte: + b = encodeInt64(b, int64(realVal)) + case int8: + b = encodeInt64(b, int64(realVal)) + case int16: + b = encodeInt64(b, int64(realVal)) + case int: + b = encodeInt64(b, int64(realVal)) + case int64: + b = encodeInt64(b, int64(realVal)) + case int32: + b = encodeInt64(b, int64(realVal)) + case uint16: + b = append(b, uintFlag) + b = EncodeUint(b, uint64(realVal)) + case uint32: + b = append(b, uintFlag) + b = EncodeUint(b, uint64(realVal)) + case uint: + b = append(b, uintFlag) + b = EncodeUint(b, uint64(realVal)) + case uint64: + b = append(b, uintFlag) + b = EncodeUint(b, uint64(realVal)) + case string: + b = encodeBytes(b, []byte(realVal)) + case []byte: + b = encodeBytes(b, realVal) + case float32: + b = append(b, floatFlag) + b = EncodeFloat(b, float64(realVal)) + case float64: + b = append(b, floatFlag) + b = EncodeFloat(b, float64(realVal)) + case nil: + b = append(b, NilFlag) + default: + return nil, fmt.Errorf("unsupport encode type %v", realVal) + } + } + return b, nil +} + +func encodeBytes(b []byte, v []byte) []byte { + b = append(b, bytesFlag) + b = EncodeBytes(b, v) + return b +} + +func encodeInt64(b []byte, v int64) []byte { + b = append(b, intFlag) + b = EncodeInt(b, v) + return b +} + +// EncodeKey appends the encoded values to byte slice b, returns the appended +// slice. It guarantees the encoded value is in ascending order for comparison. +func EncodeMemCmpKey(b []byte, v ...interface{}) ([]byte, error) { + return memcmpEncode(b, v) +} + +func EncodeMaxKey(b []byte) ([]byte, error) { + b = append(b, maxFlag) + return b, nil +} + +func EncodeMinNotNull(b []byte) ([]byte, error) { + b = append(b, bytesFlag) + return b, nil +} + +// Decode decodes values from a byte slice generated with EncodeKey or EncodeValue +// before. +// size is the size of decoded slice. +func Decode(b []byte, size int) ([]interface{}, error) { + if len(b) < 1 { + return nil, errors.New("invalid encoded key") + } + + var ( + err error + values = make([]interface{}, 0, size) + ) + + for len(b) > 0 { + var d interface{} + b, d, err = DecodeOne(b) + if err != nil { + return nil, err + } + values = append(values, d) + } + + return values, nil +} + +func DecodeOne(b []byte) (remain []byte, d interface{}, err error) { + if len(b) < 1 { + return nil, d, errors.New("invalid encoded key") + } + flag := b[0] + b = b[1:] + switch flag { + case intFlag: + var v int64 + b, v, err = DecodeInt(b) + d = v + case uintFlag: + var v uint64 + b, v, err = DecodeUint(b) + d = v + case floatFlag: + var v float64 + b, v, err = DecodeFloat(b) + d = v + case bytesFlag: + var v []byte + b, v, err = DecodeBytes(b) + d = v + case NilFlag: + default: + return b, d, fmt.Errorf("invalid encoded key flag %v", flag) + } + if err != nil { + return b, d, err + } + return b, d, nil +} + +func CutOne(b []byte) (data []byte, remain []byte, err error) { + l, err := peek(b) + if err != nil { + return nil, nil, err + } + return b[:l], b[l:], nil +} + +func peek(b []byte) (length int, err error) { + if len(b) < 1 { + return 0, errors.New("invalid encoded key") + } + flag := b[0] + length++ + b = b[1:] + var l int + switch flag { + case NilFlag: + case intFlag, floatFlag: + l = 8 + case bytesFlag: + l, err = peekBytes(b, false) + default: + return 0, fmt.Errorf("invalid encoded key flag %v", flag) + } + if err != nil { + return 0, err + } + length += l + return +} + +func peekBytes(b []byte, reverse bool) (int, error) { + offset := 0 + for { + if len(b) < offset+encGroupSize+1 { + return 0, errors.New("insufficient bytes to decode value") + } + marker := b[offset+encGroupSize] + var padCount byte + if reverse { + padCount = marker + } else { + padCount = encMarker - marker + } + offset += encGroupSize + 1 + // When padCount is not zero, it means we get the end of the byte slice. + if padCount != 0 { + break + } + } + return offset, nil +} diff --git a/nsqd/codec/number.go b/nsqd/codec/number.go new file mode 100644 index 00000000..e74887af --- /dev/null +++ b/nsqd/codec/number.go @@ -0,0 +1,160 @@ +// Copyright 2015 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package codec + +import ( + "encoding/binary" + "errors" + "math" +) + +const signMask uint64 = 0x8000000000000000 + +func encodeIntToCmpUint(v int64) uint64 { + return uint64(v) ^ signMask +} + +func decodeCmpUintToInt(u uint64) int64 { + return int64(u ^ signMask) +} + +// EncodeInt appends the encoded value to slice b and returns the appended slice. +// EncodeInt guarantees that the encoded value is in ascending order for comparison. +func EncodeInt(b []byte, v int64) []byte { + var data [8]byte + u := encodeIntToCmpUint(v) + binary.BigEndian.PutUint64(data[:], u) + return append(b, data[:]...) +} + +// EncodeIntDesc appends the encoded value to slice b and returns the appended slice. +// EncodeIntDesc guarantees that the encoded value is in descending order for comparison. +func EncodeIntDesc(b []byte, v int64) []byte { + var data [8]byte + u := encodeIntToCmpUint(v) + binary.BigEndian.PutUint64(data[:], ^u) + return append(b, data[:]...) +} + +// DecodeInt decodes value encoded by EncodeInt before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeInt(b []byte) ([]byte, int64, error) { + if len(b) < 8 { + return nil, 0, errors.New("insufficient bytes to decode value") + } + + u := binary.BigEndian.Uint64(b[:8]) + v := decodeCmpUintToInt(u) + b = b[8:] + return b, v, nil +} + +// DecodeIntDesc decodes value encoded by EncodeInt before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeIntDesc(b []byte) ([]byte, int64, error) { + if len(b) < 8 { + return nil, 0, errors.New("insufficient bytes to decode value") + } + + u := binary.BigEndian.Uint64(b[:8]) + v := decodeCmpUintToInt(^u) + b = b[8:] + return b, v, nil +} + +// EncodeUint appends the encoded value to slice b and returns the appended slice. +// EncodeUint guarantees that the encoded value is in ascending order for comparison. +func EncodeUint(b []byte, v uint64) []byte { + var data [8]byte + binary.BigEndian.PutUint64(data[:], v) + return append(b, data[:]...) +} + +// EncodeUintDesc appends the encoded value to slice b and returns the appended slice. +// EncodeUintDesc guarantees that the encoded value is in descending order for comparison. +func EncodeUintDesc(b []byte, v uint64) []byte { + var data [8]byte + binary.BigEndian.PutUint64(data[:], ^v) + return append(b, data[:]...) +} + +// DecodeUint decodes value encoded by EncodeUint before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeUint(b []byte) ([]byte, uint64, error) { + if len(b) < 8 { + return nil, 0, errors.New("insufficient bytes to decode value") + } + + v := binary.BigEndian.Uint64(b[:8]) + b = b[8:] + return b, v, nil +} + +// DecodeUintDesc decodes value encoded by EncodeInt before. +// It returns the leftover un-decoded slice, decoded value if no error. +func DecodeUintDesc(b []byte) ([]byte, uint64, error) { + if len(b) < 8 { + return nil, 0, errors.New("insufficient bytes to decode value") + } + + data := b[:8] + v := binary.BigEndian.Uint64(data) + b = b[8:] + return b, ^v, nil +} + +func encodeFloatToCmpUint64(f float64) uint64 { + u := math.Float64bits(f) + if f >= 0 { + u |= signMask + } else { + u = ^u + } + return u +} + +func decodeCmpUintToFloat(u uint64) float64 { + if u&signMask > 0 { + u &= ^signMask + } else { + u = ^u + } + return math.Float64frombits(u) +} + +// EncodeFloat encodes a float v into a byte slice which can be sorted lexicographically later. +// EncodeFloat guarantees that the encoded value is in ascending order for comparison. +func EncodeFloat(b []byte, v float64) []byte { + u := encodeFloatToCmpUint64(v) + return EncodeUint(b, u) +} + +// DecodeFloat decodes a float from a byte slice generated with EncodeFloat before. +func DecodeFloat(b []byte) ([]byte, float64, error) { + b, u, err := DecodeUint(b) + return b, decodeCmpUintToFloat(u), err +} + +// EncodeFloatDesc encodes a float v into a byte slice which can be sorted lexicographically later. +// EncodeFloatDesc guarantees that the encoded value is in descending order for comparison. +func EncodeFloatDesc(b []byte, v float64) []byte { + u := encodeFloatToCmpUint64(v) + return EncodeUintDesc(b, u) +} + +// DecodeFloatDesc decodes a float from a byte slice generated with EncodeFloatDesc before. +func DecodeFloatDesc(b []byte) ([]byte, float64, error) { + b, u, err := DecodeUintDesc(b) + return b, decodeCmpUintToFloat(u), err +} diff --git a/nsqd/delay_queue.go b/nsqd/delay_queue.go index 5b0f391c..d0ee8361 100644 --- a/nsqd/delay_queue.go +++ b/nsqd/delay_queue.go @@ -529,6 +529,17 @@ func (q *DelayQueue) TryFixQueueEnd(vend BackendOffset, totalCnt int64) error { return err } +func (q *DelayQueue) CheckDiskQueueReadToEndOK(offset int64, seekCnt int64, endOffset BackendOffset) error { + snap := q.GetDiskQueueSnapshot(false) + defer snap.Close() + _, _, err := snap.CheckDiskQueueReadToEndOK(offset, seekCnt, endOffset) + if err != nil { + nsqLog.Warningf("check read failed at: %v, err: %s", offset, err) + return err + } + return nil +} + func (q *DelayQueue) ResetBackendWithQueueStartNoLock(queueStartOffset int64, queueStartCnt int64) error { if queueStartOffset < 0 || queueStartCnt < 0 { return errors.New("queue start should not less than 0") @@ -545,7 +556,7 @@ func (q *DelayQueue) ResetBackendWithQueueStartNoLock(queueStartOffset int64, qu return nil } -func (q *DelayQueue) GetDiskQueueSnapshot() *DiskQueueSnapshot { +func (q *DelayQueue) GetDiskQueueSnapshot(checkCommit bool) *DiskQueueSnapshot { e := q.backend.GetQueueReadEnd() start := q.backend.GetQueueReadStart() d := NewDiskQueueSnapshot(getDelayQueueBackendName(q.tname, q.partition), q.dataPath, e) diff --git a/nsqd/disk_queue_snapshot.go b/nsqd/disk_queue_snapshot.go index 01ccb7fd..5cf17166 100644 --- a/nsqd/disk_queue_snapshot.go +++ b/nsqd/disk_queue_snapshot.go @@ -41,6 +41,20 @@ func NewDiskQueueSnapshot(readFrom string, dataPath string, endInfo BackendQueue return &d } +func (d *DiskQueueSnapshot) ResetToStart() { + d.Lock() + defer d.Unlock() + + if d.exitFlag == 1 { + return + } + d.readPos = d.queueStart + if d.readFile != nil { + d.readFile.Close() + d.readFile = nil + } +} + func (d *DiskQueueSnapshot) getCurrentFileEnd(offset diskQueueOffset) (int64, error) { curFileName := d.fileName(offset.FileNum) f, err := os.Stat(curFileName) @@ -242,6 +256,11 @@ func (d *DiskQueueSnapshot) ReadRaw(size int32) ([]byte, error) { curFileName := d.fileName(d.readPos.EndOffset.FileNum) d.readFile, err = os.OpenFile(curFileName, os.O_RDONLY, 0600) if err != nil { + if d.readPos.Offset() == d.endPos.Offset() && d.readPos.EndOffset == d.endPos.EndOffset { + if os.IsNotExist(err) { + return nil, io.EOF + } + } return result, err } nsqLog.LogDebugf("DISKQUEUE snapshot(%s): readRaw() opened %s", d.readFrom, curFileName) @@ -313,6 +332,11 @@ CheckFileOpen: curFileName := d.fileName(d.readPos.EndOffset.FileNum) d.readFile, result.Err = os.OpenFile(curFileName, os.O_RDONLY, 0600) if result.Err != nil { + if d.readPos.Offset() == d.endPos.Offset() && d.readPos.EndOffset == d.endPos.EndOffset { + if os.IsNotExist(result.Err) { + result.Err = io.EOF + } + } return result } @@ -351,7 +375,7 @@ CheckFileOpen: // where a new message should begin d.readFile.Close() d.readFile = nil - result.Err = fmt.Errorf("invalid message read size (%d)", msgSize) + result.Err = fmt.Errorf("invalid message read size (%d) at offset: %v", msgSize, d.readPos) return result } @@ -378,6 +402,38 @@ CheckFileOpen: return result } +func (d *DiskQueueSnapshot) CheckDiskQueueReadToEndOK(offset int64, seekCnt int64, endOffset BackendOffset) (int64, int64, error) { + localErr := d.SeekTo(BackendOffset(offset), seekCnt) + if localErr != nil { + return offset, seekCnt, localErr + } + // read until end since it may have multi in the last batch + lastOffset := offset + lastCnt := seekCnt + for { + r := d.ReadOne() + if r.Err != nil { + // should not have eof since it will break after last read + nsqLog.Warningf("check read failed at: %v, err: %s", lastOffset, r.Err) + return lastOffset, lastCnt, r.Err + } else { + if r.Offset+r.MovedSize == endOffset { + lastOffset = int64(endOffset) + lastCnt = r.CurCnt + break + } + if r.Offset+r.MovedSize > endOffset { + err := fmt.Errorf("check read failed, unexpected end offset: %v, %v, %v", r.Offset, r.MovedSize, endOffset) + nsqLog.Warningf("%s", err) + return lastOffset, lastCnt, err + } + lastOffset = int64(r.Offset + r.MovedSize) + lastCnt = r.CurCnt + } + } + return lastOffset, lastCnt, nil +} + func (d *DiskQueueSnapshot) handleReachEnd() { if d.readFile != nil { d.readFile.Close() diff --git a/nsqd/diskqueue_reader.go b/nsqd/diskqueue_reader.go index b8a9a023..98fc56f7 100644 --- a/nsqd/diskqueue_reader.go +++ b/nsqd/diskqueue_reader.go @@ -116,6 +116,7 @@ type diskQueueReader struct { autoSkipError bool waitingMoreData int32 metaStorage IMetaStorage + kvTopic *KVTopic } func newDiskQueueReaderWithFileMeta(readFrom string, metaname string, dataPath string, maxBytesPerFile int64, @@ -123,7 +124,7 @@ func newDiskQueueReaderWithFileMeta(readFrom string, metaname string, dataPath s syncEvery int64, syncTimeout time.Duration, readEnd BackendQueueEnd, autoSkip bool) BackendQueueReader { metaStorage := &fileMetaStorage{} return newDiskQueueReader(readFrom, metaname, dataPath, maxBytesPerFile, minMsgSize, maxMsgSize, syncEvery, - syncTimeout, readEnd, autoSkip, metaStorage, true) + syncTimeout, readEnd, autoSkip, metaStorage, nil, true) } // newDiskQueue instantiates a new instance of diskQueueReader, retrieving metadata @@ -133,7 +134,7 @@ func newDiskQueueReader(readFrom string, metaname string, minMsgSize int32, maxMsgSize int32, syncEvery int64, syncTimeout time.Duration, readEnd BackendQueueEnd, autoSkip bool, - metaStorage IMetaStorage, forceReload bool) BackendQueueReader { + metaStorage IMetaStorage, kvTopic *KVTopic, forceReload bool) BackendQueueReader { d := diskQueueReader{ readFrom: readFrom, @@ -144,6 +145,7 @@ func newDiskQueueReader(readFrom string, metaname string, exitChan: make(chan int), syncEvery: syncEvery, autoSkipError: autoSkip, + kvTopic: kvTopic, readBuffer: bytes.NewBuffer(make([]byte, 0, readBufferSize)), } if metaStorage == nil { @@ -461,6 +463,18 @@ func (d *diskQueueReader) TryReadOne() (ReadResult, bool) { continue } } + if d.kvTopic != nil { + rmsg, err := d.kvTopic.GetMsgRawByCnt(dataRead.CurCnt - 1) + if err != nil { + nsqLog.Logf("reading from kv failed: %v, %v, %v", dataRead.CurCnt, dataRead.Offset, err.Error()) + dataRead.Err = err + } else { + if !bytes.Equal(rmsg, dataRead.Data) { + nsqLog.LogWarningf("reading from kv not matched: %v, %v, %v, %v", dataRead.CurCnt, dataRead.Offset, rmsg, dataRead.Data) + dataRead.Err = errors.New("kv data not matched") + } + } + } return dataRead, true } else { if nsqLog.Level() >= levellogger.LOG_DETAIL { diff --git a/nsqd/diskqueue_reader_test.go b/nsqd/diskqueue_reader_test.go index f433f1b2..d7728ed6 100644 --- a/nsqd/diskqueue_reader_test.go +++ b/nsqd/diskqueue_reader_test.go @@ -17,7 +17,7 @@ func newDiskQueueReaderWithMetaStorage(readFrom string, metaname string, dataPat syncEvery int64, syncTimeout time.Duration, readEnd BackendQueueEnd, autoSkip bool) BackendQueueReader { return newDiskQueueReader(readFrom, metaname, dataPath, maxBytesPerFile, minMsgSize, maxMsgSize, syncEvery, - syncTimeout, readEnd, autoSkip, testMetaStorage, true) + syncTimeout, readEnd, autoSkip, testMetaStorage, nil, true) } func TestDiskQueueReaderResetConfirmed(t *testing.T) { diff --git a/nsqd/engine/iterator.go b/nsqd/engine/iterator.go new file mode 100644 index 00000000..b14fe3b3 --- /dev/null +++ b/nsqd/engine/iterator.go @@ -0,0 +1,249 @@ +package engine + +import ( + "bytes" + "fmt" + "runtime" +) + +type IteratorGetter interface { + GetIterator(IteratorOpts) (Iterator, error) +} + +type Iterator interface { + Next() + Prev() + Valid() bool + Seek([]byte) + SeekForPrev([]byte) + SeekToFirst() + SeekToLast() + Close() + RefKey() []byte + Key() []byte + RefValue() []byte + Value() []byte + NoTimestamp(vt byte) +} +type emptyIterator struct { +} + +func (eit *emptyIterator) Valid() bool { + return false +} + +func (eit *emptyIterator) Next() { +} +func (eit *emptyIterator) Prev() { +} +func (eit *emptyIterator) Seek([]byte) { +} +func (eit *emptyIterator) SeekForPrev([]byte) { +} +func (eit *emptyIterator) SeekToFirst() { +} +func (eit *emptyIterator) SeekToLast() { +} +func (eit *emptyIterator) Close() { +} +func (eit *emptyIterator) RefKey() []byte { + return nil +} +func (eit *emptyIterator) Key() []byte { + return nil +} +func (eit *emptyIterator) RefValue() []byte { + return nil +} +func (eit *emptyIterator) Value() []byte { + return nil +} +func (eit *emptyIterator) NoTimestamp(vt byte) { +} + +type Range struct { + Min []byte + Max []byte + Type uint8 +} + +type Limit struct { + Offset int + Count int +} + +type IteratorOpts struct { + Range + Limit + Reverse bool + IgnoreDel bool + WithSnap bool +} + +// note: all the iterator use the prefix iterator flag. Which means it may skip the keys for different table +// prefix. +func NewDBRangeLimitIteratorWithOpts(ig IteratorGetter, opts IteratorOpts) (rit *RangeLimitedIterator, err error) { + var dbit Iterator + dbit, err = ig.GetIterator(opts) + if err != nil { + return nil, err + } + defer func() { + if e := recover(); e != nil { + dbit.Close() + buf := make([]byte, 4096) + n := runtime.Stack(buf, false) + buf = buf[0:n] + err = fmt.Errorf("init iterator panic: %s:%v", buf, e) + } + }() + if !opts.Reverse { + rit = NewRangeLimitIterator(dbit, &opts.Range, + &opts.Limit) + } else { + rit = NewRevRangeLimitIterator(dbit, &opts.Range, + &opts.Limit) + } + return +} + +// note: all the iterator use the prefix iterator flag. Which means it may skip the keys for different table +// prefix. +func NewDBRangeIteratorWithOpts(ig IteratorGetter, opts IteratorOpts) (rit *RangeLimitedIterator, err error) { + var dbit Iterator + dbit, err = ig.GetIterator(opts) + if err != nil { + return nil, err + } + defer func() { + if e := recover(); e != nil { + dbit.Close() + buf := make([]byte, 4096) + n := runtime.Stack(buf, false) + buf = buf[0:n] + err = fmt.Errorf("init iterator panic: %s:%v", buf, e) + } + }() + if !opts.Reverse { + rit = NewRangeIterator(dbit, &opts.Range) + } else { + rit = NewRevRangeIterator(dbit, &opts.Range) + } + return +} + +type RangeLimitedIterator struct { + Iterator + l Limit + r Range + // maybe step should not auto increase, we need count for actually element + step int + reverse bool +} + +func (it *RangeLimitedIterator) Valid() bool { + if it.l.Offset < 0 { + return false + } + if it.l.Count >= 0 && it.step >= it.l.Count { + return false + } + if !it.Iterator.Valid() { + return false + } + + if !it.reverse { + if it.r.Max != nil { + r := bytes.Compare(it.Iterator.RefKey(), it.r.Max) + if it.r.Type&RangeROpen > 0 { + return !(r >= 0) + } else { + return !(r > 0) + } + } + } else { + if it.r.Min != nil { + r := bytes.Compare(it.Iterator.RefKey(), it.r.Min) + if it.r.Type&RangeLOpen > 0 { + return !(r <= 0) + } else { + return !(r < 0) + } + } + } + return true +} + +func (it *RangeLimitedIterator) Next() { + it.step++ + if !it.reverse { + it.Iterator.Next() + } else { + it.Iterator.Prev() + } +} + +func NewRangeLimitIterator(i Iterator, r *Range, l *Limit) *RangeLimitedIterator { + return rangeLimitIterator(i, r, l, false) +} +func NewRevRangeLimitIterator(i Iterator, r *Range, l *Limit) *RangeLimitedIterator { + return rangeLimitIterator(i, r, l, true) +} +func NewRangeIterator(i Iterator, r *Range) *RangeLimitedIterator { + return rangeLimitIterator(i, r, &Limit{0, -1}, false) +} +func NewRevRangeIterator(i Iterator, r *Range) *RangeLimitedIterator { + return rangeLimitIterator(i, r, &Limit{0, -1}, true) +} +func rangeLimitIterator(i Iterator, r *Range, l *Limit, reverse bool) *RangeLimitedIterator { + it := &RangeLimitedIterator{ + Iterator: i, + l: *l, + r: *r, + reverse: reverse, + step: 0, + } + if l.Offset < 0 { + return it + } + if !reverse { + if r.Min == nil { + it.Iterator.SeekToFirst() + } else { + it.Iterator.Seek(r.Min) + if r.Type&RangeLOpen > 0 { + if it.Iterator.Valid() && bytes.Compare(it.Iterator.RefKey(), r.Min) <= 0 { + it.Iterator.Next() + } + } + } + } else { + if r.Max == nil { + it.Iterator.SeekToLast() + } else { + it.Iterator.SeekForPrev(r.Max) + if !it.Iterator.Valid() { + it.Iterator.SeekToFirst() + if it.Iterator.Valid() && bytes.Compare(it.Iterator.RefKey(), r.Max) == 1 { + dbLog.Infof("iterator seek to last key %v should not great than seek to max %v", it.Iterator.RefKey(), r.Max) + } + } + if r.Type&RangeROpen > 0 { + if it.Iterator.Valid() && bytes.Compare(it.Iterator.RefKey(), r.Max) >= 0 { + it.Iterator.Prev() + } + } + } + } + for i := 0; i < l.Offset; i++ { + if !it.Valid() { + break + } + if !it.reverse { + it.Iterator.Next() + } else { + it.Iterator.Prev() + } + } + return it +} diff --git a/nsqd/engine/kv.go b/nsqd/engine/kv.go new file mode 100644 index 00000000..5930779e --- /dev/null +++ b/nsqd/engine/kv.go @@ -0,0 +1,244 @@ +package engine + +import ( + "errors" + "path" + + "github.com/shirou/gopsutil/mem" + "github.com/youzan/nsq/internal/levellogger" +) + +var ( + errDBEngClosed = errors.New("db engine is closed") + errIntNumber = errors.New("invalid integer") +) + +const ( + RangeClose uint8 = 0x00 + RangeLOpen uint8 = 0x01 + RangeROpen uint8 = 0x10 + RangeOpen uint8 = 0x11 +) + +const ( + DIR_PERM = 0755 + FILE_PERM = 0644 +) + +type RefSlice interface { + // ref data + Data() []byte + Free() + // copied data if need + Bytes() []byte +} + +const ( + compactThreshold = 5000000 +) + +var dbLog = levellogger.NewLevelLogger(levellogger.LOG_INFO, levellogger.NewSimpleLog()) + +func SetLogLevel(level int32) { + dbLog.SetLevel(level) +} + +func SetLogger(level int32, logger levellogger.Logger) { + dbLog.SetLevel(level) + dbLog.Logger = logger +} + +type CRange struct { + Start []byte + Limit []byte +} + +type SharedRockConfig interface { + Destroy() + ChangeLimiter(bytesPerSec int64) +} + +type RockEngConfig struct { + DataDir string + ReadOnly bool + DataTool bool + SharedConfig SharedRockConfig + EnableTableCounter bool + AutoCompacted bool + RockOptions +} + +func NewRockConfig() *RockEngConfig { + c := &RockEngConfig{ + EnableTableCounter: true, + } + FillDefaultOptions(&c.RockOptions) + return c +} + +type RockOptions struct { + VerifyReadChecksum bool `json:"verify_read_checksum"` + BlockSize int `json:"block_size"` + BlockCache int64 `json:"block_cache"` + WriteBufferSize int `json:"write_buffer_size"` + MaxWriteBufferNumber int `json:"max_write_buffer_number"` + MinWriteBufferNumberToMerge int `json:"min_write_buffer_number_to_merge"` + Level0FileNumCompactionTrigger int `json:"level0_file_num_compaction_trigger"` + MaxBytesForLevelBase uint64 `json:"max_bytes_for_level_base"` + TargetFileSizeBase uint64 `json:"target_file_size_base"` + MaxBackgroundFlushes int `json:"max_background_flushes"` + MaxBackgroundCompactions int `json:"max_background_compactions"` + MinLevelToCompress int `json:"min_level_to_compress"` + MaxMainifestFileSize uint64 `json:"max_mainifest_file_size"` + RateBytesPerSec int64 `json:"rate_bytes_per_sec"` + BackgroundHighThread int `json:"background_high_thread,omitempty"` + BackgroundLowThread int `json:"background_low_thread,omitempty"` + AdjustThreadPool bool `json:"adjust_thread_pool,omitempty"` + UseSharedCache bool `json:"use_shared_cache,omitempty"` + UseSharedRateLimiter bool `json:"use_shared_rate_limiter,omitempty"` + DisableWAL bool `json:"disable_wal,omitempty"` + DisableMergeCounter bool `json:"disable_merge_counter,omitempty"` + OptimizeFiltersForHits bool `json:"optimize_filters_for_hits,omitempty"` + // note do not change this dynamic for existing db + LevelCompactionDynamicLevelBytes bool `json:"level_compaction_dynamic_level_bytes,omitempty"` + InsertHintFixedLen int `json:"insert_hint_fixed_len"` + EngineType string `json:"engine_type,omitempty"` +} + +func FillDefaultOptions(opts *RockOptions) { + // use large block to reduce index block size for hdd + // if using ssd, should use the default value + if opts.BlockSize <= 0 { + // for hdd use 64KB and above + // for ssd use 32KB and below + opts.BlockSize = 1024 * 8 + } + // should about 20% less than host RAM + // http://smalldatum.blogspot.com/2016/09/tuning-rocksdb-block-cache.html + if opts.BlockCache <= 0 { + v, err := mem.VirtualMemory() + if err != nil { + opts.BlockCache = 1024 * 1024 * 128 + } else { + opts.BlockCache = int64(v.Total / 100) + if opts.UseSharedCache { + opts.BlockCache *= 10 + } else { + if opts.BlockCache < 1024*1024*64 { + opts.BlockCache = 1024 * 1024 * 64 + } else if opts.BlockCache > 1024*1024*1024*8 { + opts.BlockCache = 1024 * 1024 * 1024 * 8 + } + } + } + } + // keep level0_file_num_compaction_trigger * write_buffer_size * min_write_buffer_number_tomerge = max_bytes_for_level_base to minimize write amplification + if opts.WriteBufferSize <= 0 { + opts.WriteBufferSize = 1024 * 1024 * 64 + } + if opts.MaxWriteBufferNumber <= 0 { + opts.MaxWriteBufferNumber = 6 + } + if opts.MinWriteBufferNumberToMerge <= 0 { + opts.MinWriteBufferNumberToMerge = 2 + } + if opts.Level0FileNumCompactionTrigger <= 0 { + opts.Level0FileNumCompactionTrigger = 2 + } + if opts.MaxBytesForLevelBase <= 0 { + opts.MaxBytesForLevelBase = 1024 * 1024 * 256 + } + if opts.TargetFileSizeBase <= 0 { + opts.TargetFileSizeBase = 1024 * 1024 * 64 + } + if opts.MaxBackgroundFlushes <= 0 { + opts.MaxBackgroundFlushes = 2 + } + if opts.MaxBackgroundCompactions <= 0 { + opts.MaxBackgroundCompactions = 8 + } + if opts.MinLevelToCompress <= 0 { + opts.MinLevelToCompress = 3 + } + if opts.MaxMainifestFileSize <= 0 { + opts.MaxMainifestFileSize = 1024 * 1024 * 32 + } + if opts.AdjustThreadPool { + if opts.BackgroundHighThread <= 0 { + opts.BackgroundHighThread = 2 + } + if opts.BackgroundLowThread <= 0 { + opts.BackgroundLowThread = 16 + } + } +} + +type KVCheckpoint interface { + Save(path string, notify chan struct{}) error +} + +type ICompactFilter interface { + Name() string + Filter(level int, key, value []byte) (bool, []byte) +} + +type KVEngine interface { + NewWriteBatch() WriteBatch + DefaultWriteBatch() WriteBatch + GetDataDir() string + SetMaxBackgroundOptions(maxCompact int, maxBackJobs int) error + CheckDBEngForRead(fullPath string) error + OpenEng() error + Write(wb WriteBatch) error + DeletedBeforeCompact() int64 + AddDeletedCnt(c int64) + LastCompactTime() int64 + CompactRange(rg CRange) + CompactAllRange() + DisableManualCompact(bool) + GetApproximateTotalKeyNum() int + GetApproximateKeyNum(ranges []CRange) uint64 + GetApproximateSizes(ranges []CRange, includeMem bool) []uint64 + IsClosed() bool + CloseEng() bool + FlushAll() + CloseAll() + GetStatistics() string + GetInternalStatus() map[string]interface{} + GetInternalPropertyStatus(p string) string + GetBytesNoLock(key []byte) ([]byte, error) + GetBytes(key []byte) ([]byte, error) + MultiGetBytes(keyList [][]byte, values [][]byte, errs []error) + Exist(key []byte) (bool, error) + ExistNoLock(key []byte) (bool, error) + GetRef(key []byte) (RefSlice, error) + GetRefNoLock(key []byte) (RefSlice, error) + GetValueWithOp(key []byte, op func([]byte) error) error + GetValueWithOpNoLock(key []byte, op func([]byte) error) error + DeleteFilesInRange(rg CRange) + GetIterator(opts IteratorOpts) (Iterator, error) + NewCheckpoint(printToStdoutAlso bool) (KVCheckpoint, error) + SetOptsForLogStorage() + SetCompactionFilter(ICompactFilter) +} + +func GetDataDirFromBase(engType string, base string) (string, error) { + if engType == "" || engType == "pebble" { + return path.Join(base, "pebble"), nil + } + return "", errors.New("unknown engine type for: " + engType) +} + +func NewKVEng(cfg *RockEngConfig) (KVEngine, error) { + if cfg.EngineType == "" || cfg.EngineType == "pebble" { + return NewPebbleEng(cfg) + } + return nil, errors.New("unknown engine type for: " + cfg.EngineType) +} + +func NewSharedEngConfig(cfg RockOptions) (SharedRockConfig, error) { + if cfg.EngineType == "" || cfg.EngineType == "pebble" { + return newSharedPebbleConfig(cfg), nil + } + return nil, errors.New("unknown engine type for: " + cfg.EngineType) +} diff --git a/nsqd/engine/kv_test.go b/nsqd/engine/kv_test.go new file mode 100644 index 00000000..5bb6d255 --- /dev/null +++ b/nsqd/engine/kv_test.go @@ -0,0 +1,335 @@ +package engine + +import ( + "flag" + "io/ioutil" + "os" + "path" + "strconv" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/youzan/nsq/internal/levellogger" +) + +func TestMain(m *testing.M) { + flag.Parse() + if testing.Verbose() { + SetLogLevel(int32(levellogger.LOG_DETAIL)) + } + ret := m.Run() + os.Exit(ret) +} + +func TestPebbleEngCheckpointData(t *testing.T) { + testCheckpointData(t, "pebble") +} + +func testCheckpointData(t *testing.T, engType string) { + SetLogger(0, nil) + cfg := NewRockConfig() + tmpDir, err := ioutil.TempDir("", "checkpoint_data") + assert.Nil(t, err) + t.Log(tmpDir) + defer os.RemoveAll(tmpDir) + cfg.DataDir = tmpDir + cfg.EngineType = engType + eng, err := NewKVEng(cfg) + assert.Nil(t, err) + err = eng.OpenEng() + assert.Nil(t, err) + defer eng.CloseAll() + + ck, err := eng.NewCheckpoint(false) + assert.Nil(t, err) + // test save should not block, so lastTs should be updated soon + ckpath := path.Join(tmpDir, "newCk") + os.MkdirAll(ckpath, DIR_PERM) + // since the open engine will add rocksdb as subdir, we save it to the right place + err = ck.Save(path.Join(ckpath, engType), make(chan struct{})) + assert.Nil(t, err) + + wb := eng.DefaultWriteBatch() + knum := 3 + for j := 0; j < knum; j++ { + wb.Put([]byte("test"+strconv.Itoa(j)), []byte("test"+strconv.Itoa(j))) + } + eng.Write(wb) + wb.Clear() + + ck2, err := eng.NewCheckpoint(false) + assert.Nil(t, err) + // test save should not block, so lastTs should be updated soon + ckpath2 := path.Join(tmpDir, "newCk2") + os.MkdirAll(ckpath2, DIR_PERM) + err = ck2.Save(path.Join(ckpath2, engType), make(chan struct{})) + assert.Nil(t, err) + + cfgCK := *cfg + cfgCK.DataDir = ckpath + engCK, err := NewKVEng(&cfgCK) + assert.Nil(t, err) + err = engCK.OpenEng() + assert.Nil(t, err) + defer engCK.CloseAll() + + cfgCK2 := *cfg + cfgCK2.DataDir = ckpath2 + engCK2, err := NewKVEng(&cfgCK2) + assert.Nil(t, err) + err = engCK2.OpenEng() + assert.Nil(t, err) + defer engCK2.CloseAll() + + for j := 0; j < knum; j++ { + key := []byte("test" + strconv.Itoa(j)) + origV, err := eng.GetBytes(key) + assert.Equal(t, key, origV) + v, err := engCK.GetBytes(key) + assert.Nil(t, err) + assert.Nil(t, v) + v2, err := engCK2.GetBytes(key) + assert.Nil(t, err) + assert.Equal(t, key, v2) + assert.Equal(t, origV, v2) + } + + time.Sleep(time.Second) +} + +func TestPebbleEngIterator(t *testing.T) { + testKVIterator(t, "pebble") +} + +func testKVIterator(t *testing.T, engType string) { + SetLogger(0, nil) + cfg := NewRockConfig() + tmpDir, err := ioutil.TempDir("", "iterator_data") + assert.Nil(t, err) + t.Log(tmpDir) + defer os.RemoveAll(tmpDir) + cfg.DataDir = tmpDir + cfg.EngineType = engType + eng, err := NewKVEng(cfg) + assert.Nil(t, err) + err = eng.OpenEng() + assert.Nil(t, err) + defer eng.CloseAll() + + wb := eng.NewWriteBatch() + key := []byte("test") + wb.Put(key, key) + eng.Write(wb) + wb.Clear() + v, err := eng.GetBytes(key) + assert.Nil(t, err) + assert.Equal(t, key, v) + key2 := []byte("test2") + wb.Put(key2, key2) + eng.Write(wb) + wb.Clear() + v, err = eng.GetBytes(key2) + assert.Nil(t, err) + assert.Equal(t, key2, v) + key3 := []byte("test3") + wb.Put(key3, key3) + eng.Write(wb) + wb.Clear() + v, err = eng.GetBytes(key3) + assert.Nil(t, err) + assert.Equal(t, key3, v) + key4 := []byte("test4") + wb.Put(key4, key4) + eng.Write(wb) + wb.Clear() + v, err = eng.GetBytes(key4) + assert.Nil(t, err) + assert.Equal(t, key4, v) + it, _ := eng.GetIterator(IteratorOpts{}) + defer it.Close() + + // test seek part of prefix + it.Seek([]byte("tes")) + assert.True(t, it.Valid()) + assert.Equal(t, key, it.Key()) + assert.Equal(t, key, it.Value()) + it.Seek(key) + assert.True(t, it.Valid()) + assert.Equal(t, key, it.Key()) + assert.Equal(t, key, it.Value()) + it.Seek(key2) + assert.True(t, it.Valid()) + assert.Equal(t, key2, it.Key()) + assert.Equal(t, key2, it.Value()) + it.Seek(key4) + assert.True(t, it.Valid()) + assert.Equal(t, key4, it.Key()) + assert.Equal(t, key4, it.Value()) + it.Seek([]byte("test44")) + assert.True(t, !it.Valid()) + + it.SeekToFirst() + // change value after iterator should not change the snapshot iterator? + if engType != "mem" { + // for btree, the write will be blocked while the iterator is open + // for skiplist, we do not support snapshot + wb.Put(key4, []byte(string(key4)+"update")) + eng.Write(wb) + wb.Clear() + } + + assert.True(t, it.Valid()) + assert.Equal(t, key, it.Key()) + assert.Equal(t, key, it.Value()) + it.Next() + assert.True(t, it.Valid()) + assert.Equal(t, key2, it.Key()) + assert.Equal(t, key2, it.Value()) + it.Next() + assert.True(t, it.Valid()) + assert.Equal(t, key3, it.Key()) + assert.Equal(t, key3, it.Value()) + it.Prev() + assert.True(t, it.Valid()) + assert.Equal(t, key2, it.Key()) + assert.Equal(t, key2, it.Value()) + it.SeekToLast() + assert.True(t, it.Valid()) + assert.Equal(t, key4, it.Key()) + assert.Equal(t, key4, it.Value()) + it.Prev() + assert.True(t, it.Valid()) + assert.Equal(t, key3, it.Key()) + assert.Equal(t, key3, it.Value()) + + if engType != "pebble" { + it.SeekForPrev(key3) + assert.True(t, it.Valid()) + assert.Equal(t, key3, it.Key()) + assert.Equal(t, key3, it.Value()) + } + + it.SeekForPrev([]byte("test5")) + assert.True(t, it.Valid()) + assert.Equal(t, key4, it.Key()) + assert.Equal(t, key4, it.Value()) + + it.SeekForPrev([]byte("test1")) + assert.True(t, it.Valid()) + assert.Equal(t, key, it.Key()) + assert.Equal(t, key, it.Value()) + it.Prev() + assert.True(t, !it.Valid()) +} + +func TestPebbleEngSnapshotIterator(t *testing.T) { + testKVSnapshotIterator(t, "pebble") +} + +func testKVSnapshotIterator(t *testing.T, engType string) { + SetLogger(0, nil) + cfg := NewRockConfig() + tmpDir, err := ioutil.TempDir("", "iterator_data") + assert.Nil(t, err) + t.Log(tmpDir) + defer os.RemoveAll(tmpDir) + cfg.DataDir = tmpDir + cfg.EngineType = engType + eng, err := NewKVEng(cfg) + assert.Nil(t, err) + err = eng.OpenEng() + assert.Nil(t, err) + defer eng.CloseAll() + + wb := eng.NewWriteBatch() + key := []byte("test") + wb.Put(key, key) + key2 := []byte("test2") + wb.Put(key2, key2) + key3 := []byte("test3") + wb.Put(key3, key3) + eng.Write(wb) + wb.Clear() + + it, _ := eng.GetIterator(IteratorOpts{}) + defer it.Close() + // modify after iterator snapshot + wb = eng.NewWriteBatch() + wb.Put(key2, []byte("changed")) + wb.Put(key3, []byte("changed")) + eng.Write(wb) + wb.Clear() + + it.Seek(key) + assert.True(t, it.Valid()) + assert.Equal(t, key, it.Key()) + assert.Equal(t, key, it.Value()) + it.Seek(key2) + assert.True(t, it.Valid()) + assert.Equal(t, key2, it.Key()) + assert.Equal(t, key2, it.Value()) + it.Seek(key3) + assert.True(t, it.Valid()) + assert.Equal(t, key3, it.Key()) + assert.Equal(t, key3, it.Value()) + + it2, _ := eng.GetIterator(IteratorOpts{}) + defer it2.Close() + + it2.Seek(key) + assert.True(t, it2.Valid()) + assert.Equal(t, key, it2.Key()) + assert.Equal(t, key, it2.Value()) + it2.Seek(key2) + assert.True(t, it2.Valid()) + assert.Equal(t, key2, it2.Key()) + assert.Equal(t, []byte("changed"), it2.Value()) + it2.Seek(key3) + assert.True(t, it2.Valid()) + assert.Equal(t, key3, it2.Key()) + assert.Equal(t, []byte("changed"), it2.Value()) +} + +func TestSpecialDataSeekForPebble(t *testing.T) { + testSpecialDataSeekForAnyType(t, "pebble") +} + +func testSpecialDataSeekForAnyType(t *testing.T, engType string) { + base := []byte{1, 0, 1, 0} + key := append([]byte{}, base...) + key2 := append([]byte{}, base...) + minKey := []byte{1, 0, 1} + + SetLogger(0, nil) + cfg := NewRockConfig() + tmpDir, err := ioutil.TempDir("", "iterator_data") + assert.Nil(t, err) + t.Log(tmpDir) + defer os.RemoveAll(tmpDir) + cfg.DataDir = tmpDir + cfg.EngineType = engType + eng, err := NewKVEng(cfg) + assert.Nil(t, err) + err = eng.OpenEng() + assert.Nil(t, err) + defer eng.CloseAll() + + wb := eng.NewWriteBatch() + value := []byte{1} + + wb.Put(key, value) + eng.Write(wb) + wb.Clear() + key2 = append(key2, []byte{1}...) + wb.Put(key2, value) + eng.Write(wb) + wb.Clear() + + it, _ := eng.GetIterator(IteratorOpts{}) + defer it.Close() + it.Seek(minKey) + assert.True(t, it.Valid()) + assert.Equal(t, key, it.Key()) + assert.Equal(t, value, it.Value()) +} diff --git a/nsqd/engine/pebble_eng.go b/nsqd/engine/pebble_eng.go new file mode 100644 index 00000000..7fe95bba --- /dev/null +++ b/nsqd/engine/pebble_eng.go @@ -0,0 +1,565 @@ +package engine + +import ( + "encoding/binary" + "errors" + "io" + "os" + "path" + "sync" + "sync/atomic" + "time" + + "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/bloom" + "github.com/shirou/gopsutil/mem" +) + +const ( + numOfLevels = 7 +) + +type pebbleRefSlice struct { + b []byte + c io.Closer +} + +func (rs *pebbleRefSlice) Free() { + if rs.c != nil { + rs.c.Close() + } +} + +func (rs *pebbleRefSlice) Bytes() []byte { + if rs.b == nil { + return nil + } + b := make([]byte, len(rs.b)) + copy(b, rs.b) + return b +} + +func (rs *pebbleRefSlice) Data() []byte { + return rs.b +} + +func GetRocksdbUint64(v []byte, err error) (uint64, error) { + if err != nil { + return 0, err + } else if v == nil || len(v) == 0 { + return 0, nil + } else if len(v) != 8 { + return 0, errIntNumber + } + + return binary.LittleEndian.Uint64(v), nil +} + +type Uint64AddMerger struct { + buf []byte +} + +func (m *Uint64AddMerger) MergeNewer(value []byte) error { + cur, err := GetRocksdbUint64(m.buf, nil) + if err != nil { + return err + } + vint, err := GetRocksdbUint64(value, nil) + if err != nil { + return err + } + nv := cur + vint + if m.buf == nil { + m.buf = make([]byte, 8) + } + binary.LittleEndian.PutUint64(m.buf, nv) + return nil +} + +func (m *Uint64AddMerger) MergeOlder(value []byte) error { + return m.MergeNewer(value) +} + +func (m *Uint64AddMerger) Finish(flag bool) ([]byte, io.Closer, error) { + return m.buf, nil, nil +} + +func newUint64AddMerger() *pebble.Merger { + return &pebble.Merger{ + Merge: func(key, value []byte) (pebble.ValueMerger, error) { + res := &Uint64AddMerger{} + res.MergeNewer(value) + return res, nil + }, + // the name should match the rocksdb default merge name + Name: "UInt64AddOperator", + } +} + +type sharedPebbleConfig struct { + SharedCache *pebble.Cache +} + +func newSharedPebbleConfig(opt RockOptions) *sharedPebbleConfig { + sc := &sharedPebbleConfig{} + if opt.UseSharedCache { + if opt.BlockCache <= 0 { + v, err := mem.VirtualMemory() + if err != nil { + opt.BlockCache = 1024 * 1024 * 128 * 10 + } else { + opt.BlockCache = int64(v.Total / 10) + // for index cached, we double it + opt.BlockCache *= 2 + } + } + sc.SharedCache = pebble.NewCache(opt.BlockCache) + } + return sc +} + +func (sc *sharedPebbleConfig) ChangeLimiter(bytesPerSec int64) { +} + +func (sc *sharedPebbleConfig) Destroy() { + if sc.SharedCache != nil { + sc.SharedCache.Unref() + } +} + +type PebbleEng struct { + rwmutex sync.RWMutex + cfg *RockEngConfig + eng *pebble.DB + opts *pebble.Options + wo *pebble.WriteOptions + ito *pebble.IterOptions + wb *pebbleWriteBatch + engOpened int32 + lastCompact int64 + deletedCnt int64 + quit chan struct{} +} + +func NewPebbleEng(cfg *RockEngConfig) (*PebbleEng, error) { + if len(cfg.DataDir) == 0 { + return nil, errors.New("config error") + } + + if !cfg.ReadOnly { + err := os.MkdirAll(cfg.DataDir, DIR_PERM) + if err != nil { + return nil, err + } + } + lopts := make([]pebble.LevelOptions, 0) + for l := 0; l < numOfLevels; l++ { + compress := pebble.SnappyCompression + if l <= cfg.MinLevelToCompress { + compress = pebble.NoCompression + } + filter := bloom.FilterPolicy(10) + opt := pebble.LevelOptions{ + Compression: compress, + BlockSize: cfg.BlockSize, + TargetFileSize: int64(cfg.TargetFileSizeBase), + FilterPolicy: filter, + } + opt.EnsureDefaults() + lopts = append(lopts, opt) + } + + opts := &pebble.Options{ + Levels: lopts, + MaxManifestFileSize: int64(cfg.MaxMainifestFileSize), + MemTableSize: cfg.WriteBufferSize, + MemTableStopWritesThreshold: cfg.MaxWriteBufferNumber, + LBaseMaxBytes: int64(cfg.MaxBytesForLevelBase), + L0CompactionThreshold: cfg.Level0FileNumCompactionTrigger, + MaxOpenFiles: -1, + MaxConcurrentCompactions: cfg.MaxBackgroundCompactions, + EventListener: pebble.MakeLoggingEventListener(nil), + } + opts.EventListener.WALCreated = nil + opts.EventListener.WALDeleted = nil + opts.EventListener.FlushBegin = nil + opts.EventListener.FlushEnd = nil + opts.EventListener.TableCreated = nil + opts.EventListener.TableDeleted = nil + opts.EventListener.ManifestCreated = nil + opts.EventListener.ManifestDeleted = nil + if cfg.DisableWAL { + opts.DisableWAL = true + } + // prefix search + comp := *pebble.DefaultComparer + opts.Comparer = &comp + opts.Comparer.Split = func(a []byte) int { + if len(a) <= 3 { + return len(a) + } + return 3 + } + cfg.EnableTableCounter = false + db := &PebbleEng{ + cfg: cfg, + opts: opts, + ito: &pebble.IterOptions{}, + wo: &pebble.WriteOptions{ + Sync: !cfg.DisableWAL, + }, + quit: make(chan struct{}), + } + if cfg.AutoCompacted { + go db.compactLoop() + } + + return db, nil +} + +func (pe *PebbleEng) NewWriteBatch() WriteBatch { + if pe.eng == nil { + panic("nil engine, should only get write batch after db opened") + } + return newPebbleWriteBatch(pe.eng, pe.wo) +} + +func (pe *PebbleEng) DefaultWriteBatch() WriteBatch { + if pe.wb == nil { + panic("nil write batch, should only get write batch after db opened") + } + return pe.wb +} + +func (pe *PebbleEng) GetDataDir() string { + return path.Join(pe.cfg.DataDir, "pebble") +} + +func (pe *PebbleEng) SetCompactionFilter(ICompactFilter) { +} + +func (pe *PebbleEng) SetMaxBackgroundOptions(maxCompact int, maxBackJobs int) error { + return nil +} + +func (pe *PebbleEng) compactLoop() { + ticker := time.NewTicker(time.Hour) + interval := (time.Hour / time.Second).Nanoseconds() + dbLog.Infof("start auto compact loop : %v", interval) + for { + select { + case <-pe.quit: + return + case <-ticker.C: + if (pe.DeletedBeforeCompact() > compactThreshold) && + (time.Now().Unix()-pe.LastCompactTime()) > interval { + dbLog.Infof("auto compact : %v, %v", pe.DeletedBeforeCompact(), pe.LastCompactTime()) + pe.CompactAllRange() + } + } + } +} + +func (pe *PebbleEng) CheckDBEngForRead(fullPath string) error { + ro := pe.opts.Clone() + ro.ErrorIfNotExists = true + ro.ReadOnly = true + //ro.Cache = nil + db, err := pebble.Open(fullPath, ro) + if err != nil { + return err + } + db.Close() + return nil +} + +func (pe *PebbleEng) OpenEng() error { + if !pe.IsClosed() { + dbLog.Warningf("engine already opened: %v, should close it before reopen", pe.GetDataDir()) + return errors.New("open failed since not closed") + } + pe.rwmutex.Lock() + defer pe.rwmutex.Unlock() + if pe.cfg.UseSharedCache && pe.cfg.SharedConfig != nil { + sc, ok := pe.cfg.SharedConfig.(*sharedPebbleConfig) + if ok { + pe.opts.Cache = sc.SharedCache + dbLog.Infof("using shared cache for pebble engine") + } + } else { + cache := pebble.NewCache(pe.cfg.BlockCache) + defer cache.Unref() + pe.opts.Cache = cache + } + opt := pe.opts + if pe.cfg.ReadOnly { + opt = pe.opts.Clone() + opt.ErrorIfNotExists = true + opt.ReadOnly = true + } + eng, err := pebble.Open(pe.GetDataDir(), opt) + if err != nil { + return err + } + pe.wb = newPebbleWriteBatch(eng, pe.wo) + pe.eng = eng + atomic.StoreInt32(&pe.engOpened, 1) + dbLog.Infof("engine opened: %v", pe.GetDataDir()) + return nil +} + +func (pe *PebbleEng) Write(wb WriteBatch) error { + return wb.Commit() +} + +func (pe *PebbleEng) DeletedBeforeCompact() int64 { + return atomic.LoadInt64(&pe.deletedCnt) +} + +func (pe *PebbleEng) AddDeletedCnt(c int64) { + atomic.AddInt64(&pe.deletedCnt, c) +} + +func (pe *PebbleEng) LastCompactTime() int64 { + return atomic.LoadInt64(&pe.lastCompact) +} + +func (pe *PebbleEng) CompactRange(rg CRange) { + atomic.StoreInt64(&pe.lastCompact, time.Now().Unix()) + atomic.StoreInt64(&pe.deletedCnt, 0) + pe.rwmutex.RLock() + closed := pe.IsClosed() + pe.rwmutex.RUnlock() + if closed { + return + } + pe.eng.Compact(rg.Start, rg.Limit) +} + +func (pe *PebbleEng) CompactAllRange() { + pe.CompactRange(CRange{}) +} + +func (pe *PebbleEng) DisableManualCompact(disable bool) { +} + +func (pe *PebbleEng) GetApproximateTotalKeyNum() int { + return 0 +} + +func (pe *PebbleEng) GetApproximateKeyNum(ranges []CRange) uint64 { + return 0 +} + +func (pe *PebbleEng) SetOptsForLogStorage() { + return +} + +func (pe *PebbleEng) GetApproximateSizes(ranges []CRange, includeMem bool) []uint64 { + pe.rwmutex.RLock() + defer pe.rwmutex.RUnlock() + sizeList := make([]uint64, len(ranges)) + if pe.IsClosed() { + return sizeList + } + for i, r := range ranges { + sizeList[i], _ = pe.eng.EstimateDiskUsage(r.Start, r.Limit) + } + return sizeList +} + +func (pe *PebbleEng) IsClosed() bool { + if atomic.LoadInt32(&pe.engOpened) == 0 { + return true + } + return false +} + +func (pe *PebbleEng) FlushAll() { + if pe.cfg.DisableWAL { + pe.eng.Flush() + } +} + +func (pe *PebbleEng) CloseEng() bool { + pe.rwmutex.Lock() + defer pe.rwmutex.Unlock() + if pe.eng != nil { + if atomic.CompareAndSwapInt32(&pe.engOpened, 1, 0) { + if pe.wb != nil { + pe.wb.Destroy() + } + if pe.cfg.DisableWAL { + pe.eng.Flush() + } + pe.eng.Close() + dbLog.Infof("engine closed: %v", pe.GetDataDir()) + return true + } + } + return false +} + +func (pe *PebbleEng) CloseAll() { + select { + case <-pe.quit: + default: + close(pe.quit) + } + pe.CloseEng() +} + +func (pe *PebbleEng) GetStatistics() string { + pe.rwmutex.RLock() + defer pe.rwmutex.RUnlock() + if pe.IsClosed() { + return "" + } + return pe.eng.Metrics().String() +} + +func (pe *PebbleEng) GetInternalStatus() map[string]interface{} { + s := make(map[string]interface{}) + s["internal"] = pe.GetStatistics() + return s +} + +func (pe *PebbleEng) GetInternalPropertyStatus(p string) string { + return p +} + +func (pe *PebbleEng) GetBytesNoLock(key []byte) ([]byte, error) { + val, err := pe.GetRefNoLock(key) + if err != nil { + return nil, err + } + if val == nil { + return nil, nil + } + defer val.Free() + if val.Data() == nil { + return nil, nil + } + return val.Bytes(), nil +} + +func (pe *PebbleEng) GetBytes(key []byte) ([]byte, error) { + pe.rwmutex.RLock() + defer pe.rwmutex.RUnlock() + if pe.IsClosed() { + return nil, errDBEngClosed + } + return pe.GetBytesNoLock(key) +} + +func (pe *PebbleEng) MultiGetBytes(keyList [][]byte, values [][]byte, errs []error) { + pe.rwmutex.RLock() + defer pe.rwmutex.RUnlock() + if pe.IsClosed() { + for i, _ := range errs { + errs[i] = errDBEngClosed + } + return + } + for i, k := range keyList { + values[i], errs[i] = pe.GetBytesNoLock(k) + } +} + +func (pe *PebbleEng) Exist(key []byte) (bool, error) { + pe.rwmutex.RLock() + defer pe.rwmutex.RUnlock() + if pe.IsClosed() { + return false, errDBEngClosed + } + return pe.ExistNoLock(key) +} + +func (pe *PebbleEng) ExistNoLock(key []byte) (bool, error) { + val, err := pe.GetRefNoLock(key) + if err != nil { + return false, err + } + if val == nil { + return false, nil + } + ok := val.Data() != nil + val.Free() + return ok, nil +} + +func (pe *PebbleEng) GetRefNoLock(key []byte) (RefSlice, error) { + val, c, err := pe.eng.Get(key) + if err != nil && err != pebble.ErrNotFound { + return nil, err + } + return &pebbleRefSlice{b: val, c: c}, nil +} + +func (pe *PebbleEng) GetRef(key []byte) (RefSlice, error) { + pe.rwmutex.RLock() + defer pe.rwmutex.RUnlock() + if pe.IsClosed() { + return nil, errDBEngClosed + } + return pe.GetRefNoLock(key) +} + +func (pe *PebbleEng) GetValueWithOp(key []byte, + op func([]byte) error) error { + pe.rwmutex.RLock() + defer pe.rwmutex.RUnlock() + if pe.IsClosed() { + return errDBEngClosed + } + return pe.GetValueWithOpNoLock(key, op) +} + +func (pe *PebbleEng) GetValueWithOpNoLock(key []byte, + op func([]byte) error) error { + val, err := pe.GetRef(key) + if err != nil { + return err + } + if val == nil { + return op(nil) + } + defer val.Free() + return op(val.Data()) +} + +func (pe *PebbleEng) DeleteFilesInRange(rg CRange) { + return +} + +func (pe *PebbleEng) GetIterator(opts IteratorOpts) (Iterator, error) { + dbit, err := newPebbleIterator(pe, opts) + if err != nil { + return nil, err + } + return dbit, nil +} + +func (pe *PebbleEng) NewCheckpoint(printToStdoutAlso bool) (KVCheckpoint, error) { + return &pebbleEngCheckpoint{ + pe: pe, + }, nil +} + +type pebbleEngCheckpoint struct { + pe *PebbleEng +} + +func (pck *pebbleEngCheckpoint) Save(path string, notify chan struct{}) error { + pck.pe.rwmutex.RLock() + defer pck.pe.rwmutex.RUnlock() + if pck.pe.IsClosed() { + return errDBEngClosed + } + if notify != nil { + time.AfterFunc(time.Millisecond*20, func() { + close(notify) + }) + } + return pck.pe.eng.Checkpoint(path) +} diff --git a/nsqd/engine/pebble_eng_test.go b/nsqd/engine/pebble_eng_test.go new file mode 100644 index 00000000..9ff7c709 --- /dev/null +++ b/nsqd/engine/pebble_eng_test.go @@ -0,0 +1,199 @@ +package engine + +import ( + "fmt" + "io/ioutil" + "os" + "path" + "runtime" + "strconv" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestPebbleCheckpointDuringWrite(t *testing.T) { + SetLogger(0, nil) + cfg := NewRockConfig() + tmpDir, err := ioutil.TempDir("", "checkpoint") + assert.Nil(t, err) + t.Log(tmpDir) + defer os.RemoveAll(tmpDir) + cfg.DataDir = tmpDir + eng, err := NewPebbleEng(cfg) + assert.Nil(t, err) + err = eng.OpenEng() + assert.Nil(t, err) + defer eng.CloseAll() + + start := time.Now() + stopC := make(chan struct{}) + var wg sync.WaitGroup + lastTs := time.Now().UnixNano() + wg.Add(1) + go func() { + defer wg.Done() + for { + select { + case <-stopC: + return + default: + } + begin := time.Now() + ck, err := eng.NewCheckpoint(false) + assert.Nil(t, err) + // test save should not block, so lastTs should be updated soon + ckpath := path.Join(tmpDir, "newCk") + err = ck.Save(ckpath, make(chan struct{})) + assert.Nil(t, err) + atomic.StoreInt64(&lastTs, time.Now().UnixNano()) + if time.Since(begin) > time.Second*5 { + t.Logf("checkpoint too long: %v, %v", begin, time.Since(begin)) + } + os.RemoveAll(ckpath) + time.Sleep(time.Millisecond * 100) + } + }() + bigV := make([]byte, 8000) + var panicTimer *time.Timer + for { + for i := 0; i < 100; i++ { + wb := eng.DefaultWriteBatch() + for j := 0; j < 100; j++ { + wb.Put([]byte("test"+strconv.Itoa(i+j)), []byte("test"+strconv.Itoa(i+j)+string(bigV))) + } + eng.Write(wb) + wb.Clear() + tn := time.Now().UnixNano() + if atomic.LoadInt64(&lastTs)+time.Second.Nanoseconds()*30 < tn { + t.Errorf("failed to wait checkpoint update: %v, %v", atomic.LoadInt64(&lastTs), tn) + panicTimer = time.AfterFunc(time.Second*10, func() { + buf := make([]byte, 1024*1024) + runtime.Stack(buf, true) + fmt.Printf("%s", buf) + panic("failed") + }) + break + } + if time.Since(start) > time.Minute { + break + } + } + if panicTimer != nil { + break + } + time.Sleep(time.Microsecond * 10) + if time.Since(start) > time.Minute { + break + } + } + close(stopC) + t.Log("waiting stop") + wg.Wait() + t.Log("waiting stopped") + if panicTimer != nil { + panicTimer.Stop() + } + time.Sleep(time.Second * 2) +} + +func TestPebbleReopenAndCheck(t *testing.T) { + SetLogger(0, nil) + cfg := NewRockConfig() + tmpDir, err := ioutil.TempDir("", "checkpoint") + assert.Nil(t, err) + t.Log(tmpDir) + defer os.RemoveAll(tmpDir) + cfg.DataDir = tmpDir + pe, err := NewPebbleEng(cfg) + err = pe.OpenEng() + assert.Nil(t, err) + wb := pe.DefaultWriteBatch() + wb.Put([]byte("test"), []byte("test")) + err = pe.Write(wb) + assert.Nil(t, err) + wb.Clear() + ck, _ := pe.NewCheckpoint(false) + err = ck.Save(path.Join(tmpDir, "cktmp"), make(chan struct{})) + assert.Nil(t, err) + + err = pe.CheckDBEngForRead(path.Join(tmpDir, "cktmp")) + assert.Nil(t, err) + pe.CloseEng() + + pe.OpenEng() + time.Sleep(time.Second * 10) + + pe.CloseEng() + pe.OpenEng() + + pe.CloseAll() + time.Sleep(time.Second * 10) +} + +func TestPebbleSharedCacheForMulti(t *testing.T) { + SetLogger(0, nil) + cfg := NewRockConfig() + tmpDir, err := ioutil.TempDir("", "checkpoint") + assert.Nil(t, err) + t.Log(tmpDir) + defer os.RemoveAll(tmpDir) + cfg.DataDir = path.Join(tmpDir, "test") + cfg.UseSharedCache = true + cfg.SharedConfig = newSharedPebbleConfig(cfg.RockOptions) + pe, err := NewPebbleEng(cfg) + assert.Nil(t, err) + err = pe.OpenEng() + assert.Nil(t, err) + defer pe.CloseAll() + + wb := pe.DefaultWriteBatch() + wb.Put([]byte("test"), []byte("test")) + err = pe.Write(wb) + wb.Clear() + assert.Nil(t, err) + + pe.eng.Flush() + + cfg2 := cfg + cfg2.DataDir = path.Join(tmpDir, "test2") + pe2, err := NewPebbleEng(cfg2) + assert.Nil(t, err) + err = pe2.OpenEng() + assert.Nil(t, err) + assert.Equal(t, pe.opts.Cache, pe2.opts.Cache) + defer pe2.CloseAll() + + wb2 := pe2.DefaultWriteBatch() + wb2.Put([]byte("test"), []byte("test2")) + err = pe2.Write(wb2) + assert.Nil(t, err) + wb2.Clear() + pe2.eng.Flush() + + v1, err := pe.GetBytes([]byte("test")) + assert.Nil(t, err) + assert.Equal(t, []byte("test"), v1) + v2, err := pe2.GetBytes([]byte("test")) + assert.Nil(t, err) + assert.Equal(t, []byte("test2"), v2) + + wb = pe.DefaultWriteBatch() + wb.Put([]byte("test"), []byte("test")) + err = pe.Write(wb) + assert.Nil(t, err) + wb.Clear() + pe.eng.Flush() + + v1, err = pe.GetBytes([]byte("test")) + assert.Nil(t, err) + assert.Equal(t, []byte("test"), v1) + v2, err = pe2.GetBytes([]byte("test")) + assert.Nil(t, err) + assert.Equal(t, []byte("test2"), v2) + + time.Sleep(time.Second * 10) +} diff --git a/nsqd/engine/pebble_iter.go b/nsqd/engine/pebble_iter.go new file mode 100644 index 00000000..f747dced --- /dev/null +++ b/nsqd/engine/pebble_iter.go @@ -0,0 +1,116 @@ +package engine + +import ( + "github.com/cockroachdb/pebble" +) + +type pebbleIterator struct { + *pebble.Iterator + db *PebbleEng + opt *pebble.IterOptions + snap *pebble.Snapshot + removeTsType byte +} + +// low_bound is inclusive +// upper bound is exclusive +func newPebbleIterator(db *PebbleEng, opts IteratorOpts) (*pebbleIterator, error) { + db.rwmutex.RLock() + if db.IsClosed() { + db.rwmutex.RUnlock() + return nil, errDBEngClosed + } + upperBound := opts.Max + lowerBound := opts.Min + if opts.Type&RangeROpen <= 0 && upperBound != nil { + // range right not open, we need inclusive the max, + // however upperBound is exclusive + upperBound = append(upperBound, 0) + } + + opt := &pebble.IterOptions{} + opt.LowerBound = lowerBound + opt.UpperBound = upperBound + dbit := &pebbleIterator{ + db: db, + opt: opt, + } + + if opts.WithSnap { + dbit.snap = db.eng.NewSnapshot() + dbit.Iterator = dbit.snap.NewIter(opt) + } else { + dbit.Iterator = db.eng.NewIter(opt) + } + return dbit, nil +} + +func (it *pebbleIterator) Next() { + it.Iterator.Next() +} + +func (it *pebbleIterator) Prev() { + it.Iterator.Prev() +} + +func (it *pebbleIterator) Seek(key []byte) { + it.Iterator.SeekGE(key) +} + +func (it *pebbleIterator) SeekForPrev(key []byte) { + it.Iterator.SeekLT(key) +} + +func (it *pebbleIterator) SeekToFirst() { + it.Iterator.First() +} + +func (it *pebbleIterator) SeekToLast() { + it.Iterator.Last() +} + +func (it *pebbleIterator) Valid() bool { + if it.Iterator.Error() != nil { + return false + } + return it.Iterator.Valid() +} + +// the bytes returned will be freed after next +func (it *pebbleIterator) RefKey() []byte { + return it.Iterator.Key() +} + +func (it *pebbleIterator) Key() []byte { + v := it.Iterator.Key() + vv := make([]byte, len(v)) + copy(vv, v) + return vv +} + +// the bytes returned will be freed after next +func (it *pebbleIterator) RefValue() []byte { + v := it.Iterator.Value() + return v +} + +func (it *pebbleIterator) Value() []byte { + v := it.RefValue() + vv := make([]byte, len(v)) + copy(vv, v) + return vv +} + +func (it *pebbleIterator) NoTimestamp(vt byte) { + it.removeTsType = vt +} + +func (it *pebbleIterator) Close() { + if it.Iterator != nil { + it.Iterator.Close() + } + if it.snap != nil { + it.snap.Close() + } + it.db.rwmutex.RUnlock() +} diff --git a/nsqd/engine/writebatch.go b/nsqd/engine/writebatch.go new file mode 100644 index 00000000..7804faea --- /dev/null +++ b/nsqd/engine/writebatch.go @@ -0,0 +1,60 @@ +package engine + +import ( + "errors" + + "github.com/cockroachdb/pebble" +) + +type WriteBatch interface { + Destroy() + Clear() + DeleteRange(start, end []byte) + Delete(key []byte) + Put(key []byte, value []byte) + Commit() error +} + +type pebbleWriteBatch struct { + wb *pebble.Batch + wo *pebble.WriteOptions + db *pebble.DB +} + +func newPebbleWriteBatch(db *pebble.DB, wo *pebble.WriteOptions) *pebbleWriteBatch { + return &pebbleWriteBatch{ + wb: db.NewBatch(), + wo: wo, + db: db, + } +} + +func (wb *pebbleWriteBatch) Destroy() { + wb.wb.Close() +} + +func (wb *pebbleWriteBatch) Clear() { + wb.wb.Close() + wb.wb = wb.db.NewBatch() + // TODO: reuse it + //wb.wb.Reset() +} + +func (wb *pebbleWriteBatch) DeleteRange(start, end []byte) { + wb.wb.DeleteRange(start, end, wb.wo) +} + +func (wb *pebbleWriteBatch) Delete(key []byte) { + wb.wb.Delete(key, wb.wo) +} + +func (wb *pebbleWriteBatch) Put(key []byte, value []byte) { + wb.wb.Set(key, value, wb.wo) +} + +func (wb *pebbleWriteBatch) Commit() error { + if wb.db == nil || wb.wo == nil { + return errors.New("nil db or options") + } + return wb.db.Apply(wb.wb, wb.wo) +} diff --git a/nsqd/kv_topic.go b/nsqd/kv_topic.go new file mode 100644 index 00000000..269ddaea --- /dev/null +++ b/nsqd/kv_topic.go @@ -0,0 +1,946 @@ +package nsqd + +import ( + "bytes" + "encoding/binary" + "errors" + "fmt" + "os" + "path" + "sync" + "sync/atomic" + + "github.com/youzan/nsq/internal/levellogger" + "github.com/youzan/nsq/nsqd/codec" + "github.com/youzan/nsq/nsqd/engine" +) + +const ( + SepStart = ":" + SepStop = ";" + maxBatchSize = 1000 +) + +const ( + TopicPrefix = "topic" + // TopicPrefix + topicFullName + SepStart + SomeOtherPrefix + TopicMetaPrefix = "metav1" + TopicMsgIDPrefix = "msgid" + TopicTagPrefix = "tag" + TopicTraceKeyPrefix = "tracekey" + TopicMsgOffsetPrefix = "msgoffset" + TopicMsgCntPrefix = "msgcnt" + TopicMsgTsPrefix = "msgts" +) + +var ( + ErrMsgNotFoundInIndex = errors.New("message not found in index") + errInvalidEncodedData = errors.New("invalid encoded data") + errBatchSizeLimit = errors.New("batch size limit exceeded") + errInvalidLimit = errors.New("invalid limit, should be >0") +) + +func getKVKeyForTopicBegin(fullName string) []byte { + startKey := make([]byte, 0, len(TopicPrefix)+len(fullName)+2+4) + startKey, _ = codec.EncodeMemCmpKey(startKey[:0], TopicPrefix, SepStart, fullName, SepStart) + return startKey +} + +func getKVKeyForTopicEnd(fullName string) []byte { + endKey := make([]byte, 0, len(TopicPrefix)+len(fullName)+2+4) + endKey, _ = codec.EncodeMemCmpKey(endKey[:0], TopicPrefix, SepStart, fullName, SepStop) + return endKey +} + +func getKVKeyForTopicMeta(fullName string) []byte { + key := make([]byte, 0, len(TopicPrefix)+len(fullName)+len(TopicMetaPrefix)+2+5) + key, _ = codec.EncodeMemCmpKey(key[:0], TopicPrefix, SepStart, fullName, SepStart, TopicMetaPrefix) + return key +} + +func encodeTopicMetaValue(offset int64, cnt int64) []byte { + buf := make([]byte, 0, 8+8+2) + buf, _ = codec.EncodeMemCmpKey(buf, offset, cnt) + return buf +} + +func decodeTopicMetaValue(buf []byte) (int64, int64, error) { + vals, err := codec.Decode(buf, 2) + if err != nil { + return 0, 0, err + } + if len(vals) < 2 { + return 0, 0, errInvalidEncodedData + } + offset := vals[0].(int64) + cnt := vals[1].(int64) + return offset, cnt, nil +} + +func getKVKeyForMsgID(fullName string, msgid MessageID) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+len(TopicMsgIDPrefix)+8+3+7) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicMsgIDPrefix, SepStart, uint64(msgid)) + return keyBuf +} + +func getKVKeyForMsgIDEnd(fullName string) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+len(TopicMsgIDPrefix)+3+6) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicMsgIDPrefix, SepStop) + return keyBuf +} + +func encodeMsgIDOffsetCntValue(msgid MessageID, offset, cnt int64) []byte { + idBuf := make([]byte, 0, 3*(8+1)) + idBuf, _ = codec.EncodeMemCmpKey(idBuf, uint64(msgid), offset, cnt) + return idBuf +} + +func decodeMsgIDOffsetCntValue(buf []byte) (MessageID, int64, int64, error) { + vals, err := codec.Decode(buf, 3) + if err != nil { + return 0, 0, 0, err + } + if len(vals) < 3 { + return 0, 0, 0, errInvalidEncodedData + } + id := vals[0].(uint64) + offset := vals[1].(int64) + cnt := vals[2].(int64) + return MessageID(id), offset, cnt, nil +} + +func getKVKeyForMsgTrace(fullName string, msgid MessageID, traceID uint64) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+len(TopicTraceKeyPrefix)+8+8+4+9) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicTraceKeyPrefix, SepStart, traceID, SepStart, uint64(msgid)) + return keyBuf +} + +func getKVKeyForMsgTraceIDEnd(fullName string, traceID uint64) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+4+len(TopicTraceKeyPrefix)+8+4+8) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicTraceKeyPrefix, SepStart, traceID, SepStop) + return keyBuf +} + +func getKVKeyForMsgTraceEnd(fullName string) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+len(TopicTraceKeyPrefix)+3+6) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicTraceKeyPrefix, SepStop) + return keyBuf +} + +func decodeMsgTraceKey(buf []byte) (string, uint64, MessageID, error) { + var tname string + var tid uint64 + var msgid MessageID + vals, err := codec.Decode(buf, 9) + if err != nil { + return tname, tid, msgid, err + } + if len(vals) != 9 { + return tname, tid, msgid, errInvalidEncodedData + } + tname = string(vals[2].([]byte)) + tid = vals[6].(uint64) + msgid = MessageID(vals[8].(uint64)) + return tname, tid, msgid, nil +} + +func getKVKeyForMsgOffset(fullName string, msgStartOffset int64, msgEndOffset int64) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+4+len(TopicTraceKeyPrefix)+8+8+4+9) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicMsgOffsetPrefix, SepStart, msgStartOffset, SepStart, msgEndOffset) + return keyBuf +} + +func getKVKeyForMsgOffsetEnd(fullName string) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+len(TopicTraceKeyPrefix)+3+6) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicMsgOffsetPrefix, SepStop) + return keyBuf +} + +func decodeMsgOffsetKey(b []byte) (int64, int64, error) { + vals, err := codec.Decode(b, 9) + if err != nil { + return 0, 0, err + } + if len(vals) < 9 { + return 0, 0, errInvalidEncodedData + } + start := vals[6].(int64) + end := vals[8].(int64) + return start, end, nil +} + +func encodeMsgOffsetValue(msgid MessageID, cnt int64, ts int64) []byte { + valueBuf := make([]byte, 0, 8*3+3) + valueBuf, _ = codec.EncodeMemCmpKey(valueBuf[:0], uint64(msgid), cnt, ts) + return valueBuf +} + +func decodeMsgOffsetValue(buf []byte) (MessageID, int64, int64, error) { + var msgid MessageID + var cnt int64 + var ts int64 + vals, err := codec.Decode(buf, 3) + if err != nil { + return msgid, cnt, ts, err + } + if len(vals) != 3 { + return msgid, cnt, ts, errInvalidEncodedData + } + msgid = MessageID(vals[0].(uint64)) + cnt = vals[1].(int64) + ts = vals[2].(int64) + return msgid, cnt, ts, nil +} + +func getKVKeyForMsgCnt(fullName string, msgCnt int64) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+len(TopicTraceKeyPrefix)+8+3+7) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicMsgCntPrefix, SepStart, msgCnt) + return keyBuf +} +func getKVKeyForMsgCntEnd(fullName string) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+len(TopicTraceKeyPrefix)+3+6) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicMsgCntPrefix, SepStop) + return keyBuf +} + +func getKVKeyForMsgTs(fullName string, msgid MessageID, ts int64) []byte { + keyBuf := make([]byte, 0, len(TopicPrefix)+len(fullName)+len(TopicTraceKeyPrefix)+8+8+4+9) + keyBuf, _ = codec.EncodeMemCmpKey(keyBuf[:0], TopicPrefix, SepStart, fullName, SepStart, TopicMsgTsPrefix, SepStart, ts, SepStart, uint64(msgid)) + return keyBuf +} + +type KVTopic struct { + kvEng engine.KVEngine + tname string + fullName string + partition int + putBuffer bytes.Buffer + bp sync.Pool + isExt int32 + tpLog *levellogger.LevelLogger + lastOffset int64 + lastCnt int64 + magicCode int64 + defaultWB engine.WriteBatch +} + +func NewKVTopic(topicName string, part int, opt *Options) *KVTopic { + return NewKVTopicWithExt(topicName, part, false, opt) +} + +func NewKVTopicWithExt(topicName string, part int, ext bool, opt *Options) *KVTopic { + dataPath := path.Join(opt.DataPath, topicName) + err := os.MkdirAll(dataPath, 0755) + if err != nil { + nsqLog.LogErrorf("failed to create directory %s: %s ", dataPath, err) + return nil + } + backendName := getBackendName(topicName, part) + cfg := engine.NewRockConfig() + cfg.DisableWAL = true + cfg.MaxWriteBufferNumber = 4 + cfg.DataDir = path.Join(dataPath, backendName) + eng, err := engine.NewKVEng(cfg) + if err != nil { + nsqLog.LogErrorf("failed to create engine: %s ", err) + return nil + } + err = eng.OpenEng() + if err != nil { + nsqLog.LogErrorf("failed to open engine: %s ", err) + return nil + } + return NewKVTopicWithEngine(topicName, part, ext, eng) +} + +func NewKVTopicWithEngine(topicName string, part int, ext bool, eng engine.KVEngine) *KVTopic { + if eng == nil { + return nil + } + if part > MAX_TOPIC_PARTITION { + return nil + } + t := &KVTopic{ + tname: topicName, + partition: part, + putBuffer: bytes.Buffer{}, + } + t.fullName = GetTopicFullName(t.tname, t.partition) + t.tpLog = nsqLog.WrappedWithPrefix("["+t.fullName+"]", 0) + if ext { + t.setExt() + } + + t.bp.New = func() interface{} { + return &bytes.Buffer{} + } + + t.kvEng = eng + t.defaultWB = eng.NewWriteBatch() + offset, cnt, err := t.GetTopicMeta() + if err != nil { + t.tpLog.LogErrorf("failed to init topic meta: %s ", err) + return nil + } + t.lastOffset = offset + t.lastCnt = cnt + return t +} + +func (t *KVTopic) setExt() { + atomic.StoreInt32(&t.isExt, 1) +} + +func (t *KVTopic) IsExt() bool { + return atomic.LoadInt32(&t.isExt) == 1 +} + +func (t *KVTopic) BufferPoolGet(capacity int) *bytes.Buffer { + b := t.bp.Get().(*bytes.Buffer) + b.Reset() + b.Grow(capacity) + return b +} + +func (t *KVTopic) BufferPoolPut(b *bytes.Buffer) { + t.bp.Put(b) +} + +func (t *KVTopic) GetFullName() string { + return t.fullName +} + +func (t *KVTopic) GetTopicName() string { + return t.tname +} + +func (t *KVTopic) GetTopicPart() int { + return t.partition +} + +func (t *KVTopic) GetTopicMeta() (int64, int64, error) { + key := getKVKeyForTopicMeta(t.fullName) + v, err := t.kvEng.GetBytes(key) + if err != nil { + return 0, 0, err + } + if v == nil { + return 0, 0, nil + } + return decodeTopicMetaValue(v) +} + +func (t *KVTopic) saveTopicMetaInBatch(wb engine.WriteBatch, offset int64, cnt int64) { + key := getKVKeyForTopicMeta(t.fullName) + buf := encodeTopicMetaValue(offset, cnt) + wb.Put(key, buf) +} + +// search the first msg [offset, end] +func (t *KVTopic) getMsgIDCntTsStartFromOffset(vend int64) (MessageID, int64, int64, int64, error) { + // tag info or trace key info currently not cleaned + minKey := getKVKeyForMsgOffset(t.fullName, vend, 0) + maxKey := getKVKeyForMsgOffsetEnd(t.fullName) + itopts := engine.IteratorOpts{} + itopts.Min = minKey + itopts.Max = maxKey + itopts.Type = engine.RangeClose + it, err := t.kvEng.GetIterator(itopts) + if err != nil { + return 0, 0, 0, 0, err + } + defer it.Close() + it.SeekToFirst() + if !it.Valid() { + // maybe no data before vend while sync data from leader on replica + return 0, 0, 0, 0, ErrMsgNotFoundInIndex + } + v := it.RefValue() + if v == nil { + return 0, 0, 0, 0, ErrMsgNotFoundInIndex + } + start, _, err := decodeMsgOffsetKey(it.RefKey()) + if err != nil { + return 0, 0, 0, 0, err + } + id, cnt, ts, err := decodeMsgOffsetValue(v) + return id, cnt, ts, start, err +} + +// search the last msg at [begin, offset) +// return id, cnt, ts, realoffset +func (t *KVTopic) getMsgIDCntTsLessThanOffset(vend int64) (MessageID, int64, int64, int64, error) { + // tag info or trace key info currently not cleaned + minKey := getKVKeyForMsgOffset(t.fullName, 0, 0) + maxKey := getKVKeyForMsgOffset(t.fullName, vend, 0) + itopts := engine.IteratorOpts{} + itopts.Min = minKey + itopts.Max = maxKey + itopts.Type = engine.RangeOpen + it, err := t.kvEng.GetIterator(itopts) + if err != nil { + return 0, 0, 0, 0, err + } + defer it.Close() + it.SeekToLast() + if !it.Valid() { + // maybe no data before vend while sync data from leader on replica + return 0, 0, 0, 0, nil + } + v := it.RefValue() + if v == nil { + return 0, 0, 0, 0, ErrMsgNotFoundInIndex + } + start, _, err := decodeMsgOffsetKey(it.RefKey()) + if err != nil { + return 0, 0, 0, 0, err + } + id, cnt, ts, err := decodeMsgOffsetValue(v) + return id, cnt, ts, start, err +} + +func (t *KVTopic) getMsgIDCntTsAtOffset(vend int64) (MessageID, int64, int64, error) { + // tag info or trace key info currently not cleaned + minKey := getKVKeyForMsgOffset(t.fullName, vend, 0) + maxKey := getKVKeyForMsgOffset(t.fullName, vend+1, 0) + itopts := engine.IteratorOpts{} + itopts.Min = minKey + itopts.Max = maxKey + itopts.Type = engine.RangeOpen + it, err := t.kvEng.GetIterator(itopts) + if err != nil { + return 0, 0, 0, err + } + defer it.Close() + it.SeekToFirst() + if !it.Valid() { + // maybe no data before vend while sync data from leader on replica + return 0, 0, 0, nil + } + v := it.RefValue() + if v == nil { + return 0, 0, 0, ErrMsgNotFoundInIndex + } + return decodeMsgOffsetValue(v) +} + +func (t *KVTopic) ResetBackendEnd(vend BackendOffset, totalCnt int64) error { + msgid, msgCnt, _, foundOffset, err := t.getMsgIDCntTsStartFromOffset(int64(vend)) + if err != nil { + return err + } + if foundOffset != int64(vend) { + t.tpLog.Warningf("reset offset %v not matched with db at offset: %v, id: %v", vend, foundOffset, msgid) + } + // the message at offset and greater than offset should be removed + // so the left count is the count index at offset + if msgCnt != totalCnt { + t.tpLog.Warningf("total count %v not matched with db %v at offset: %v, id: %v", totalCnt, msgCnt, vend, msgid) + } + minKey := getKVKeyForMsgOffset(t.fullName, foundOffset, 0) + maxKey := getKVKeyForMsgOffsetEnd(t.fullName) + wb := t.kvEng.NewWriteBatch() + defer wb.Destroy() + wb.DeleteRange(minKey, maxKey) + + minKey = getKVKeyForMsgID(t.fullName, MessageID(msgid)) + maxKey = getKVKeyForMsgIDEnd(t.fullName) + wb.DeleteRange(minKey, maxKey) + + minKey = getKVKeyForMsgCnt(t.fullName, msgCnt) + maxKey = getKVKeyForMsgCntEnd(t.fullName) + wb.DeleteRange(minKey, maxKey) + + t.saveTopicMetaInBatch(wb, foundOffset, msgCnt) + //minKey = getKVKeyForMsgTs(t.fullName, MessageID(msgid), msgTs) + //maxKey = getKVKeyForMsgTsEnd(t.fullName) + //wb.DeleteRange(minKey, maxKey) + err = wb.Commit() + if err != nil { + return err + } + t.lastOffset = foundOffset + t.lastCnt = msgCnt + return nil +} + +// PutMessage writes a Message to the queue +func (t *KVTopic) PutMessage(m *Message) (int32, BackendQueueEnd, error) { + endOffset, writeBytes, dendCnt, err := t.put(m, 0) + if err != nil { + return writeBytes, nil, err + } + dn := &diskQueueEndInfo{totalMsgCnt: dendCnt} + dn.virtualEnd = endOffset + return writeBytes, dn, err +} + +// this raw data include size header +func (t *KVTopic) PutRawDataOnReplica(rawData []byte, offset BackendOffset, checkSize int64, msgNum int32) (BackendQueueEnd, error) { + wend := t.lastOffset + if wend != int64(offset) { + t.tpLog.LogErrorf("topic write offset mismatch: %v, %v", offset, wend) + return nil, ErrWriteOffsetMismatch + } + if len(rawData) < 4 { + return nil, fmt.Errorf("invalid raw message data: %v", rawData) + } + wb := t.defaultWB + defer wb.Clear() + // batched + leftBuf := rawData + wsizeTotal := int32(0) + var diskEnd diskQueueEndInfo + diskEnd.totalMsgCnt = t.lastCnt + diskEnd.virtualEnd = BackendOffset(t.lastOffset) + for { + if len(leftBuf) < 4 { + return nil, fmt.Errorf("invalid raw message data: %v", rawData) + } + sz := int32(binary.BigEndian.Uint32(leftBuf[:4])) + if sz <= 0 || sz > MAX_POSSIBLE_MSG_SIZE { + // this file is corrupt and we have no reasonable guarantee on + // where a new message should begin + return nil, fmt.Errorf("invalid message read size (%d)", sz) + } + wend, wsize, err := t.putBatchedRaw(wb, int64(diskEnd.Offset()), diskEnd.TotalMsgCnt(), leftBuf[4:sz+4], 0) + if err != nil { + return nil, err + } + msgNum-- + wsizeTotal += wsize + diskEnd.virtualEnd = wend + diskEnd.totalMsgCnt = diskEnd.totalMsgCnt + 1 + leftBuf = leftBuf[sz+4:] + if len(leftBuf) == 0 { + break + } + } + if checkSize > 0 && int64(wsizeTotal) != checkSize { + return nil, fmt.Errorf("batch message size mismatch: %v vs %v", checkSize, wsizeTotal) + } + if msgNum != 0 { + return nil, fmt.Errorf("should have the same message number in raw: %v", msgNum) + } + t.saveTopicMetaInBatch(wb, int64(diskEnd.Offset()), diskEnd.TotalMsgCnt()) + err := wb.Commit() + if err != nil { + return nil, err + } + t.lastOffset = int64(diskEnd.Offset()) + t.lastCnt = diskEnd.TotalMsgCnt() + return &diskEnd, nil +} + +func (t *KVTopic) PutMessages(msgs []*Message) (BackendOffset, int32, BackendQueueEnd, error) { + wb := t.defaultWB + defer wb.Clear() + firstOffset := BackendOffset(t.lastOffset) + var diskEnd diskQueueEndInfo + diskEnd.totalMsgCnt = t.lastCnt + diskEnd.virtualEnd = BackendOffset(t.lastOffset) + batchBytes := int32(0) + for _, m := range msgs { + offset, bytes, err := t.putBatched(wb, int64(diskEnd.Offset()), diskEnd.TotalMsgCnt(), m, 0) + if err != nil { + return firstOffset, batchBytes, &diskEnd, err + } + diskEnd.totalMsgCnt = diskEnd.totalMsgCnt + 1 + diskEnd.virtualEnd = offset + batchBytes += bytes + } + t.saveTopicMetaInBatch(wb, int64(diskEnd.Offset()), diskEnd.TotalMsgCnt()) + err := wb.Commit() + if err != nil { + return firstOffset, batchBytes, &diskEnd, err + } + t.lastOffset = int64(diskEnd.Offset()) + t.lastCnt = diskEnd.TotalMsgCnt() + return firstOffset, batchBytes, &diskEnd, nil +} + +func (t *KVTopic) putBatched(wb engine.WriteBatch, lastOffset int64, lastCnt int64, m *Message, checkSize int64) (BackendOffset, int32, error) { + var writeEnd BackendOffset + t.putBuffer.Reset() + wsize, err := m.WriteTo(&t.putBuffer, t.IsExt()) + if err != nil { + return writeEnd, 0, err + } + // there are 4bytes data length on disk. + if checkSize > 0 && wsize+4 != checkSize { + return writeEnd, 0, fmt.Errorf("message write size mismatch %v vs %v", checkSize, wsize+4) + } + writeEnd = BackendOffset(lastOffset) + BackendOffset(wsize+4) + keyBuf := getKVKeyForMsgID(t.fullName, m.ID) + wb.Put(keyBuf, t.putBuffer.Bytes()) + valBuf := encodeMsgIDOffsetCntValue(m.ID, lastOffset, lastCnt) + if m.TraceID > 0 { + keyBuf = getKVKeyForMsgTrace(t.fullName, m.ID, m.TraceID) + wb.Put(keyBuf, valBuf) + } + keyBuf = getKVKeyForMsgOffset(t.fullName, lastOffset, int64(writeEnd)) + valueBuf := encodeMsgOffsetValue(m.ID, lastCnt, m.Timestamp) + wb.Put(keyBuf, valueBuf) + keyBuf = getKVKeyForMsgCnt(t.fullName, lastCnt) + wb.Put(keyBuf, valBuf) + keyBuf = getKVKeyForMsgTs(t.fullName, m.ID, m.Timestamp) + wb.Put(keyBuf, valBuf) + return writeEnd, int32(wsize + 4), nil +} + +// note: the rawdata should not include the size header +func (t *KVTopic) putBatchedRaw(wb engine.WriteBatch, lastOffset int64, lastCnt int64, rawData []byte, checkSize int64) (BackendOffset, int32, error) { + var writeEnd BackendOffset + wsize := len(rawData) + // there are 4bytes data length on disk. + if checkSize > 0 && int64(wsize+4) != checkSize { + return writeEnd, 0, fmt.Errorf("message write size mismatch %v vs %v", checkSize, wsize+4) + } + m, err := DecodeMessage(rawData, t.IsExt()) + // note, if the origin message is not ext, we should not write the ext to the data on replica + if err != nil { + return writeEnd, 0, err + } + if m.ID <= 0 { + return writeEnd, 0, fmt.Errorf("message data invalid") + } + writeEnd = BackendOffset(lastOffset) + BackendOffset(wsize+4) + keyBuf := getKVKeyForMsgID(t.fullName, m.ID) + wb.Put(keyBuf, rawData) + valBuf := encodeMsgIDOffsetCntValue(m.ID, lastOffset, lastCnt) + if m.TraceID > 0 { + keyBuf = getKVKeyForMsgTrace(t.fullName, m.ID, m.TraceID) + wb.Put(keyBuf, valBuf) + } + keyBuf = getKVKeyForMsgOffset(t.fullName, lastOffset, int64(writeEnd)) + valueBuf := encodeMsgOffsetValue(m.ID, lastCnt, m.Timestamp) + wb.Put(keyBuf, valueBuf) + keyBuf = getKVKeyForMsgCnt(t.fullName, lastCnt) + wb.Put(keyBuf, valBuf) + keyBuf = getKVKeyForMsgTs(t.fullName, m.ID, m.Timestamp) + wb.Put(keyBuf, valBuf) + return writeEnd, int32(wsize + 4), nil +} + +// this raw has no size header +func (t *KVTopic) putRaw(rawData []byte, offset BackendOffset, checkSize int64) (BackendOffset, int32, int64, error) { + wend := t.lastOffset + if wend != int64(offset) { + t.tpLog.LogErrorf("topic write offset mismatch: %v, %v", offset, wend) + return 0, 0, 0, ErrWriteOffsetMismatch + } + wb := t.defaultWB + defer wb.Clear() + + var writeCnt int64 + writeEnd, wsize, err := t.putBatchedRaw(wb, t.lastOffset, t.lastCnt, rawData, checkSize) + if err != nil { + return writeEnd, wsize, writeCnt, err + } + writeCnt = t.lastCnt + 1 + t.saveTopicMetaInBatch(wb, int64(writeEnd), writeCnt) + // add tag if ext has tag info + err = wb.Commit() + if err != nil { + return writeEnd, 0, writeCnt, err + } + t.lastOffset = int64(writeEnd) + t.lastCnt = writeCnt + return writeEnd, int32(wsize), writeCnt, nil +} + +func (t *KVTopic) put(m *Message, checkSize int64) (BackendOffset, int32, int64, error) { + wb := t.defaultWB + defer wb.Clear() + + var writeCnt int64 + writeEnd, wsize, err := t.putBatched(wb, t.lastOffset, t.lastCnt, m, checkSize) + if err != nil { + return writeEnd, wsize, writeCnt, err + } + writeCnt = t.lastCnt + 1 + t.saveTopicMetaInBatch(wb, int64(writeEnd), writeCnt) + // add tag if ext has tag info + err = wb.Commit() + if err != nil { + return writeEnd, 0, writeCnt, err + } + t.lastOffset = int64(writeEnd) + t.lastCnt = writeCnt + return writeEnd, int32(wsize), writeCnt, nil +} + +// Delete empties the topic and all its channels and closes +func (t *KVTopic) Delete() error { + return t.exit(true) +} + +// Close persists all outstanding topic data and closes all its channels +func (t *KVTopic) Close() error { + return t.exit(false) +} + +func (t *KVTopic) exit(deleted bool) error { + if deleted { + // empty the queue (deletes the backend files, too) + t.Empty() + } + + t.defaultWB.Destroy() + return nil +} + +func (t *KVTopic) Empty() error { + t.tpLog.Logf("TOPIC empty") + startKey := getKVKeyForTopicBegin(t.fullName) + endKey := getKVKeyForTopicEnd(t.fullName) + wb := t.kvEng.NewWriteBatch() + defer wb.Destroy() + // note the meta will be deleted also + // maybe we can padding magic code in the topic name, so + // we can auto lazy delete (recreate with magic code changed) + wb.DeleteRange(startKey, endKey) + err := wb.Commit() + if err != nil { + return err + } + t.lastOffset = 0 + t.lastCnt = 0 + return nil +} + +// maybe should return the cleaned offset to allow commit log clean +func (t *KVTopic) TryCleanOldData(retentionSize int64, cleanEndInfo BackendQueueOffset, maxCleanOffset BackendOffset) error { + // clean the data that has been consumed and keep the retention policy + if cleanEndInfo == nil || cleanEndInfo.Offset()+BackendOffset(retentionSize) >= maxCleanOffset { + if cleanEndInfo != nil { + t.tpLog.Infof("clean topic data at position: %v could not exceed max clean end: %v", + cleanEndInfo, maxCleanOffset) + } + return nil + } + // clean data old then cleanEndInfo + + return t.CleanBackendWithQueueStart(int64(cleanEndInfo.Offset())) +} + +func (t *KVTopic) ResetBackendWithQueueStart(queueStartOffset int64, totalCnt int64) error { + t.tpLog.Warningf("reset the topic backend with queue start: %v, %v", queueStartOffset, totalCnt) + err := t.Empty() + if err != nil { + return err + } + wb := t.kvEng.NewWriteBatch() + defer wb.Destroy() + t.saveTopicMetaInBatch(wb, queueStartOffset, totalCnt) + err = wb.Commit() + if err != nil { + return err + } + t.lastOffset = queueStartOffset + t.lastCnt = totalCnt + return nil +} + +func (t *KVTopic) CleanBackendWithQueueStart(queueStartOffset int64) error { + t.tpLog.Infof("clean with queue start: %v", queueStartOffset) + // delete the data old than queueStartOffset + // tag info or trace key info currently not cleaned + msgid, msgCnt, msgTs, _, err := t.getMsgIDCntTsLessThanOffset(queueStartOffset) + if err != nil { + return err + } + minKey := getKVKeyForMsgOffset(t.fullName, int64(0), 0) + maxKey := getKVKeyForMsgOffset(t.fullName, int64(queueStartOffset), 0) + wb := t.kvEng.NewWriteBatch() + defer wb.Destroy() + wb.DeleteRange(minKey, maxKey) + + minKey = getKVKeyForMsgID(t.fullName, MessageID(0)) + maxKey = getKVKeyForMsgID(t.fullName, MessageID(msgid)) + wb.DeleteRange(minKey, maxKey) + wb.Delete(maxKey) + + minKey = getKVKeyForMsgCnt(t.fullName, 0) + maxKey = getKVKeyForMsgCnt(t.fullName, msgCnt) + wb.DeleteRange(minKey, maxKey) + wb.Delete(maxKey) + + minKey = getKVKeyForMsgTs(t.fullName, 0, 0) + maxKey = getKVKeyForMsgTs(t.fullName, MessageID(msgid), msgTs) + wb.DeleteRange(minKey, maxKey) + wb.Delete(maxKey) + err = wb.Commit() + return err +} + +func (t *KVTopic) GetMsgRawByID(id MessageID) ([]byte, error) { + key := getKVKeyForMsgID(t.fullName, id) + v, err := t.kvEng.GetBytes(key) + if err != nil { + return nil, err + } + if v == nil { + return nil, ErrMsgNotFoundInIndex + } + return v, nil +} + +func (t *KVTopic) GetMsgByID(id MessageID) (*Message, error) { + v, err := t.GetMsgRawByID(id) + if err != nil { + return nil, err + } + return DecodeMessage(v, t.IsExt()) +} + +func (t *KVTopic) GetMsgByTraceID(tid uint64, limit int) ([]*Message, error) { + if limit > maxBatchSize { + return nil, errBatchSizeLimit + } + ids := make([]MessageID, 0, 3) + err := func() error { + minKey := getKVKeyForMsgTrace(t.fullName, 0, tid) + maxKey := getKVKeyForMsgTraceIDEnd(t.fullName, tid) + itopts := engine.IteratorOpts{} + itopts.Min = minKey + itopts.Max = maxKey + itopts.Type = engine.RangeOpen + it, err := t.kvEng.GetIterator(itopts) + if err != nil { + return err + } + defer it.Close() + it.SeekToFirst() + for ; it.Valid(); it.Next() { + k := it.RefKey() + _, ftid, msgid, err := decodeMsgTraceKey(k) + if err != nil { + continue + } + if ftid > tid { + break + } + if ftid != tid { + continue + } + ids = append(ids, msgid) + if len(ids) >= limit { + break + } + } + return nil + }() + if err != nil { + return nil, err + } + msgs := make([]*Message, 0, len(ids)) + for _, msgid := range ids { + m, err := t.GetMsgByID(msgid) + if err != nil { + continue + } + msgs = append(msgs, m) + } + return msgs, nil +} + +func (t *KVTopic) GetMsgByOffset(offset int64) (*Message, error) { + msgid, _, _, err := t.getMsgIDCntTsAtOffset(offset) + if err != nil { + return nil, err + } + return t.GetMsgByID(MessageID(msgid)) +} + +func (t *KVTopic) GetMsgIDOffsetByCnt(cnt int64) (MessageID, int64, error) { + key := getKVKeyForMsgCnt(t.fullName, cnt) + v, err := t.kvEng.GetBytes(key) + if err != nil { + return 0, 0, err + } + if v == nil { + return 0, 0, ErrMsgNotFoundInIndex + } + id, offset, _, err := decodeMsgIDOffsetCntValue(v) + if err != nil { + return 0, 0, err + } + return id, offset, nil +} + +func (t *KVTopic) GetMsgRawByCnt(cnt int64) ([]byte, error) { + id, _, err := t.GetMsgIDOffsetByCnt(cnt) + if err != nil { + return nil, err + } + return t.GetMsgRawByID(id) +} + +func (t *KVTopic) GetMsgByCnt(cnt int64) (*Message, int64, error) { + id, offset, err := t.GetMsgIDOffsetByCnt(cnt) + if err != nil { + return nil, offset, err + } + msg, err := t.GetMsgByID(id) + return msg, offset, err +} + +func (t *KVTopic) GetMsgByTime(ts int64, limit int) ([]*Message, error) { + if limit > maxBatchSize { + return nil, errBatchSizeLimit + } + return nil, nil +} + +func (t *KVTopic) PullMsgByCntFrom(cnt int64, limit int64) ([]*Message, error) { + if limit > maxBatchSize { + return nil, errBatchSizeLimit + } + if limit <= 0 { + return nil, errInvalidLimit + } + id, _, err := t.GetMsgIDOffsetByCnt(cnt) + if err != nil { + if err == ErrMsgNotFoundInIndex { + return nil, nil + } + return nil, err + } + _, lastCnt, err := t.GetTopicMeta() + if err != nil { + return nil, err + } + if lastCnt <= cnt { + t.tpLog.Warningf("total count %v less than count %v, but have cnt key: %v", t.lastCnt, cnt, id) + return nil, nil + } + minKey := getKVKeyForMsgID(t.fullName, id) + maxKey := getKVKeyForMsgIDEnd(t.fullName) + itopts := engine.IteratorOpts{} + itopts.Min = minKey + itopts.Max = maxKey + itopts.Type = engine.RangeClose + it, err := t.kvEng.GetIterator(itopts) + if err != nil { + return nil, err + } + defer it.Close() + it.SeekToFirst() + sz := limit + if lastCnt-cnt < sz { + sz = t.lastCnt - cnt + } + msgs := make([]*Message, 0, sz) + for ; it.Valid(); it.Next() { + m, err := DecodeMessage(it.Value(), t.IsExt()) + if err != nil { + return nil, err + } + msgs = append(msgs, m) + if int64(len(msgs)) >= limit { + break + } + } + return msgs, nil +} diff --git a/nsqd/kv_topic_test.go b/nsqd/kv_topic_test.go new file mode 100644 index 00000000..44586206 --- /dev/null +++ b/nsqd/kv_topic_test.go @@ -0,0 +1,331 @@ +package nsqd + +import ( + "bytes" + "encoding/binary" + "fmt" + "io/ioutil" + "os" + "strconv" + "testing" + "time" + + "github.com/youzan/nsq/internal/ext" + "github.com/youzan/nsq/internal/test" +) + +func testKVTopicWriteRead(t *testing.T, replica bool) { + opts := NewOptions() + tmpDir, err := ioutil.TempDir("", fmt.Sprintf("nsq-test-%d", time.Now().UnixNano())) + if err != nil { + panic(err) + } + opts.DataPath = tmpDir + defer os.RemoveAll(tmpDir) + kvt := NewKVTopicWithExt("test-kv-topic", 0, true, opts) + defer kvt.kvEng.CloseAll() + defer kvt.Close() + + test.NotNil(t, kvt) + offset, totalCnt, err := kvt.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(0), offset) + test.Equal(t, int64(0), totalCnt) + singleSize := int32(47) + msgs := make([]*Message, 0) + msgCnt := 6 + rawMsgData := bytes.NewBuffer(make([]byte, 0, int(singleSize)*msgCnt)) + for i := 0; i < msgCnt; i++ { + m := NewMessageWithExt(MessageID(i+1), make([]byte, 10), ext.JSON_HEADER_EXT_VER, []byte("tes"+strconv.Itoa(i))) + if i <= msgCnt-2 { + m.TraceID = uint64(i + 1) + } + msgs = append(msgs, m) + binary.Write(rawMsgData, binary.BigEndian, singleSize-4) + m.WriteTo(rawMsgData, kvt.IsExt()) + } + test.Equal(t, int(singleSize)*msgCnt, len(rawMsgData.Bytes())) + var wsize int32 + var end BackendQueueEnd + if replica { + end, err = kvt.PutRawDataOnReplica(rawMsgData.Bytes()[:singleSize], BackendOffset(kvt.lastOffset), int64(singleSize), 1) + } else { + wsize, end, err = kvt.PutMessage(msgs[0]) + } + t.Log(err) + test.Nil(t, err) + if !replica { + test.Equal(t, singleSize, wsize) + } + test.Equal(t, end.Offset(), BackendOffset(singleSize)) + test.Equal(t, end.TotalMsgCnt(), int64(1)) + + offset, totalCnt, err = kvt.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(singleSize), offset) + test.Equal(t, int64(1), totalCnt) + + var wfirst BackendOffset + if replica { + end, err = kvt.PutRawDataOnReplica(rawMsgData.Bytes()[singleSize:], BackendOffset(kvt.lastOffset), int64(singleSize*int32(msgCnt-1)), int32(msgCnt-1)) + } else { + wfirst, wsize, end, err = kvt.PutMessages(msgs[1:]) + } + test.Nil(t, err) + if !replica { + test.Equal(t, int32(singleSize*int32(msgCnt-1)), wsize) + test.Equal(t, BackendOffset(singleSize), wfirst) + } + test.Equal(t, BackendOffset(int(singleSize)*msgCnt), end.Offset()) + test.Equal(t, int64(msgCnt), end.TotalMsgCnt()) + test.Equal(t, int64(end.Offset()), kvt.lastOffset) + test.Equal(t, end.TotalMsgCnt(), kvt.lastCnt) + + offset, totalCnt, err = kvt.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(msgCnt*int(singleSize)), offset) + test.Equal(t, int64(msgCnt), totalCnt) + + // test reopen + kvt.Close() + kvt.kvEng.CloseAll() + kvt = NewKVTopicWithExt("test-kv-topic", 0, true, opts) + test.NotNil(t, kvt) + offset, totalCnt, err = kvt.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(msgCnt*int(singleSize)), offset) + test.Equal(t, int64(msgCnt), totalCnt) + + for i := 0; i < msgCnt; i++ { + dbMsg, err := kvt.GetMsgByID(MessageID(i + 1)) + test.Nil(t, err) + test.Equal(t, msgs[i].ID, dbMsg.ID) + test.Equal(t, msgs[i].Body, dbMsg.Body) + test.Equal(t, msgs[i].TraceID, dbMsg.TraceID) + test.Equal(t, msgs[i].ExtBytes, dbMsg.ExtBytes) + dbMsg, err = kvt.GetMsgByOffset(int64(i * int(singleSize))) + test.Nil(t, err) + test.Equal(t, msgs[i].ID, dbMsg.ID) + test.Equal(t, msgs[i].Body, dbMsg.Body) + test.Equal(t, msgs[i].TraceID, dbMsg.TraceID) + test.Equal(t, msgs[i].ExtBytes, dbMsg.ExtBytes) + dbMsg, _, err = kvt.GetMsgByCnt(int64(i)) + test.Nil(t, err) + test.Equal(t, msgs[i].ID, dbMsg.ID) + test.Equal(t, msgs[i].Body, dbMsg.Body) + test.Equal(t, msgs[i].TraceID, dbMsg.TraceID) + test.Equal(t, msgs[i].ExtBytes, dbMsg.ExtBytes) + dbmsgs, err := kvt.GetMsgByTraceID(uint64(i+1), 2) + test.Nil(t, err) + if i <= msgCnt-2 { + test.Equal(t, 1, len(dbmsgs)) + test.Equal(t, msgs[i].ID, dbmsgs[0].ID) + test.Equal(t, msgs[i].Body, dbmsgs[0].Body) + test.Equal(t, msgs[i].TraceID, dbmsgs[0].TraceID) + test.Equal(t, msgs[i].ExtBytes, dbmsgs[0].ExtBytes) + } else { + test.Equal(t, 0, len(dbmsgs)) + } + } + + for i := 0; i < msgCnt+2; i++ { + for j := 1; j < msgCnt+2; j++ { + dbmsgs, err := kvt.PullMsgByCntFrom(int64(i), int64(j)) + test.Nil(t, err) + expectCnt := 0 + if i < msgCnt { + if i+j < msgCnt { + expectCnt = j + } else { + expectCnt = msgCnt - i + } + } + t.Logf("begin cnt: %v, limit: %v", i, j) + test.Equal(t, expectCnt, len(dbmsgs)) + for mi, m := range dbmsgs { + test.Equal(t, m.ID, msgs[i+mi].ID) + test.Equal(t, m.Body, msgs[i+mi].Body) + test.Equal(t, m.TraceID, msgs[i+mi].TraceID) + test.Equal(t, m.ExtBytes, msgs[i+mi].ExtBytes) + } + } + } + kvt.Empty() + offset, totalCnt, err = kvt.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(0), offset) + test.Equal(t, int64(0), totalCnt) + dbmsgs, err := kvt.PullMsgByCntFrom(0, int64(msgCnt)) + test.Nil(t, err) + test.Equal(t, 0, len(dbmsgs)) +} + +func TestKVTopicWriteRead(t *testing.T) { + testKVTopicWriteRead(t, false) +} + +func TestKVTopicWriteReadReplica(t *testing.T) { + testKVTopicWriteRead(t, true) +} + +func TestKVTopicWriteRawData(t *testing.T) { + opts := NewOptions() + tmpDir, err := ioutil.TempDir("", fmt.Sprintf("nsq-test-%d", time.Now().UnixNano())) + if err != nil { + panic(err) + } + opts.DataPath = tmpDir + defer os.RemoveAll(tmpDir) + kvt := NewKVTopicWithExt("test-kv-topic-raw", 0, true, opts) + defer kvt.kvEng.CloseAll() + defer kvt.Close() + test.NotNil(t, kvt) + singleSize := int32(47) + m := NewMessageWithExt(1, make([]byte, 10), ext.JSON_HEADER_EXT_VER, []byte("tes1")) + m.TraceID = 1 + var end BackendQueueEnd + b := bytes.NewBuffer(make([]byte, 0, singleSize)) + binary.Write(b, binary.BigEndian, singleSize-4) + m.WriteTo(b, true) + end, err = kvt.PutRawDataOnReplica(b.Bytes(), BackendOffset(kvt.lastOffset), int64(singleSize), 1) + t.Log(err) + test.Nil(t, err) + test.Equal(t, end.Offset(), BackendOffset(singleSize)) + test.Equal(t, end.TotalMsgCnt(), int64(1)) + msgs := make([]*Message, 0) + m2 := NewMessageWithExt(2, make([]byte, 10), ext.JSON_HEADER_EXT_VER, []byte("tes2")) + m2.TraceID = 2 + m3 := NewMessageWithExt(3, make([]byte, 10), ext.JSON_HEADER_EXT_VER, []byte("tes3")) + msgs = append(msgs, m2, m3) + b.Reset() + binary.Write(b, binary.BigEndian, singleSize-4) + m2.WriteTo(b, true) + binary.Write(b, binary.BigEndian, singleSize-4) + m3.WriteTo(b, true) + end, err = kvt.PutRawDataOnReplica(b.Bytes(), BackendOffset(kvt.lastOffset), int64(singleSize*2), 2) + test.Nil(t, err) + + test.Equal(t, BackendOffset(singleSize*3), end.Offset()) + test.Equal(t, int64(3), end.TotalMsgCnt()) + test.Equal(t, int64(end.Offset()), kvt.lastOffset) + test.Equal(t, end.TotalMsgCnt(), kvt.lastCnt) + + dbMsg, err := kvt.GetMsgByID(MessageID(1)) + test.Nil(t, err) + test.Equal(t, m.ID, dbMsg.ID) + test.Equal(t, m.Body, dbMsg.Body) + test.Equal(t, m.TraceID, dbMsg.TraceID) + test.Equal(t, m.ExtBytes, dbMsg.ExtBytes) + dbMsg, err = kvt.GetMsgByID(MessageID(2)) + test.Nil(t, err) + test.Equal(t, m2.ID, dbMsg.ID) + test.Equal(t, m2.Body, dbMsg.Body) + test.Equal(t, m2.TraceID, dbMsg.TraceID) + test.Equal(t, m2.ExtBytes, dbMsg.ExtBytes) + dbMsg, err = kvt.GetMsgByID(MessageID(3)) + test.Nil(t, err) + test.Equal(t, m3.ID, dbMsg.ID) + test.Equal(t, m3.Body, dbMsg.Body) + test.Equal(t, m3.TraceID, dbMsg.TraceID) + test.Equal(t, m3.ExtBytes, dbMsg.ExtBytes) + + offset, totalCnt, err := kvt.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(3*singleSize), offset) + test.Equal(t, int64(3), totalCnt) + // test reopen + kvt.Close() + kvt.kvEng.CloseAll() + kvt = NewKVTopic("test-kv-topic-raw", 0, opts) + test.NotNil(t, kvt) + offset, totalCnt, err = kvt.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(3*singleSize), offset) + test.Equal(t, int64(3), totalCnt) +} + +func TestKVTopicResetStartEnd(t *testing.T) { + opts := NewOptions() + tmpDir, err := ioutil.TempDir("", fmt.Sprintf("nsq-test-%d", time.Now().UnixNano())) + if err != nil { + panic(err) + } + opts.DataPath = tmpDir + defer os.RemoveAll(tmpDir) + kvt := NewKVTopicWithExt("test-kv-topic-reset", 0, true, opts) + defer kvt.Close() + test.NotNil(t, kvt) + singleSize := int32(47) + msgs := make([]*Message, 0) + msgCnt := 6 + for i := 0; i < msgCnt; i++ { + m := NewMessageWithExt(MessageID(i+1), make([]byte, 10), ext.JSON_HEADER_EXT_VER, []byte("tes"+strconv.Itoa(i))) + if i <= msgCnt-2 { + m.TraceID = uint64(i + 1) + } + msgs = append(msgs, m) + } + var wsize int32 + var end BackendQueueEnd + wsize, end, err = kvt.PutMessage(msgs[0]) + test.Nil(t, err) + test.Equal(t, singleSize, wsize) + test.Equal(t, end.Offset(), BackendOffset(singleSize)) + test.Equal(t, end.TotalMsgCnt(), int64(1)) + + _, wsize, end, err = kvt.PutMessages(msgs[1:]) + test.Nil(t, err) + test.Equal(t, BackendOffset(int(singleSize)*msgCnt), end.Offset()) + test.Equal(t, int64(msgCnt), end.TotalMsgCnt()) + + offset, totalCnt, err := kvt.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(msgCnt*int(singleSize)), offset) + test.Equal(t, int64(msgCnt), totalCnt) + + err = kvt.CleanBackendWithQueueStart(int64(singleSize) * 2) + test.Nil(t, err) + err = kvt.ResetBackendEnd(BackendOffset(singleSize)*BackendOffset(msgCnt-2), int64(msgCnt-2)) + test.Nil(t, err) + + offset, totalCnt, err = kvt.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64((msgCnt-2)*int(singleSize)), offset) + test.Equal(t, int64(msgCnt-2), totalCnt) + dbmsgs, err := kvt.PullMsgByCntFrom(2, int64(msgCnt)) + test.Nil(t, err) + test.Equal(t, int(msgCnt)-4, len(dbmsgs)) + for i := 0; i < msgCnt; i++ { + if i < msgCnt-4 { + test.Equal(t, msgs[i+2].ID, dbmsgs[i].ID) + test.Equal(t, msgs[i+2].TraceID, dbmsgs[i].TraceID) + } + dbmsg, err := kvt.GetMsgByOffset(int64(i * int(singleSize))) + t.Logf("%v:%v", i, dbmsg) + if i < 2 || i >= msgCnt-2 { + test.Equal(t, ErrMsgNotFoundInIndex, err) + } else { + test.Nil(t, err) + test.Equal(t, msgs[i].ID, dbmsg.ID) + test.Equal(t, msgs[i].TraceID, dbmsg.TraceID) + } + dbmsg, err = kvt.GetMsgByID(msgs[i].ID) + t.Logf("%v:%v", i, dbmsg) + if i < 2 || i >= msgCnt-2 { + test.Equal(t, ErrMsgNotFoundInIndex, err) + } else { + test.Nil(t, err) + test.Equal(t, msgs[i].ID, dbmsg.ID) + test.Equal(t, msgs[i].TraceID, dbmsg.TraceID) + } + dbmsg, _, err = kvt.GetMsgByCnt(int64(i)) + t.Logf("%v:%v", i, dbmsg) + if i < 2 || i >= msgCnt-2 { + test.Equal(t, ErrMsgNotFoundInIndex, err) + } else { + test.Nil(t, err) + test.Equal(t, msgs[i].ID, dbmsg.ID) + test.Equal(t, msgs[i].TraceID, dbmsg.TraceID) + } + } +} diff --git a/nsqd/meta_storage.go b/nsqd/meta_storage.go index ae7c79a2..e33fddb5 100644 --- a/nsqd/meta_storage.go +++ b/nsqd/meta_storage.go @@ -16,7 +16,7 @@ import ( "time" "github.com/absolute8511/bolt" - "github.com/spaolacci/murmur3" + "github.com/twmb/murmur3" "github.com/youzan/nsq/internal/util" ) diff --git a/nsqd/nsqd.go b/nsqd/nsqd.go index f1b2a2d7..c3717524 100644 --- a/nsqd/nsqd.go +++ b/nsqd/nsqd.go @@ -16,7 +16,7 @@ import ( "time" "github.com/bitly/go-simplejson" - "github.com/spaolacci/murmur3" + "github.com/twmb/murmur3" "github.com/youzan/nsq/internal/clusterinfo" "github.com/youzan/nsq/internal/dirlock" "github.com/youzan/nsq/internal/http_api" @@ -25,6 +25,7 @@ import ( "github.com/youzan/nsq/internal/statsd" "github.com/youzan/nsq/internal/util" "github.com/youzan/nsq/internal/version" + "github.com/youzan/nsq/nsqd/engine" ) const ( @@ -95,9 +96,11 @@ type NSQD struct { persistClosed chan struct{} persistWaitGroup util.WaitGroupWrapper metaStorage IMetaStorage + kvTopicStorage engine.KVEngine + sharedCfg engine.SharedRockConfig } -func New(opts *Options) *NSQD { +func New(opts *Options) (*NSQD, error) { dataPath := opts.DataPath if opts.DataPath == "" { cwd, _ := os.Getwd() @@ -107,7 +110,7 @@ func New(opts *Options) *NSQD { err := os.MkdirAll(dataPath, 0755) if err != nil { nsqLog.LogErrorf("failed to create directory: %v ", err) - os.Exit(1) + return nil, err } if opts.RetentionSizePerDay > 0 { DEFAULT_RETENTION_DAYS = int(opts.RetentionDays) @@ -144,24 +147,67 @@ func New(opts *Options) *NSQD { err = n.dl.Lock() if err != nil { nsqLog.LogErrorf("FATAL: --data-path=%s in use (possibly by another instance of nsqd: %v", dataPath, err) - os.Exit(1) + return nil, err } if opts.MaxDeflateLevel < 1 || opts.MaxDeflateLevel > 9 { nsqLog.LogErrorf("FATAL: --max-deflate-level must be [1,9]") - os.Exit(1) + return nil, errors.New("configure invalid") } if opts.ID < 0 || opts.ID >= MAX_NODE_ID { nsqLog.LogErrorf("FATAL: --worker-id must be [0,%d)", MAX_NODE_ID) - os.Exit(1) + return nil, errors.New("configure invalid") } nsqLog.Logf("broadcast option: %s, %s", opts.BroadcastAddress, opts.BroadcastInterface) n.metaStorage, err = NewShardedDBMetaStorage(path.Join(dataPath, "shared_meta")) if err != nil { nsqLog.LogErrorf("FATAL: init shared meta storage failed: %v", err.Error()) - os.Exit(1) + return nil, err + } + if opts.KVEnabled { + kvPath := path.Join(dataPath, "shared_kvdata") + err = os.MkdirAll(kvPath, 0755) + if err != nil { + nsqLog.LogErrorf("failed to create directory %s: %s ", kvPath, err) + return nil, err + } + + cfg := engine.NewRockConfig() + // we disable wal here, because we use this for index data, and we will + // auto recovery it from disk queue data + cfg.DisableWAL = true + cfg.UseSharedCache = true + cfg.UseSharedRateLimiter = true + if opts.KVMaxWriteBufferNumber > 0 { + cfg.MaxWriteBufferNumber = int(opts.KVMaxWriteBufferNumber) + } + if opts.KVWriteBufferSize > 0 { + cfg.WriteBufferSize = int(opts.KVWriteBufferSize) + } + if opts.KVBlockCache > 0 { + cfg.BlockCache = opts.KVBlockCache + } + sharedCfg, err := engine.NewSharedEngConfig(cfg.RockOptions) + if err != nil { + nsqLog.LogErrorf("failed to init engine config %v: %s ", cfg, err) + } + n.sharedCfg = sharedCfg + cfg.SharedConfig = sharedCfg + cfg.DataDir = kvPath + engine.FillDefaultOptions(&cfg.RockOptions) + eng, err := engine.NewKVEng(cfg) + if err != nil { + nsqLog.LogErrorf("failed to create engine: %s ", err) + return nil, err + } + err = eng.OpenEng() + if err != nil { + nsqLog.LogErrorf("failed to open engine: %s ", err) + return nil, err + } + n.kvTopicStorage = eng } if opts.StatsdPrefix != "" { @@ -172,7 +218,7 @@ func New(opts *Options) *NSQD { _, port, err = net.SplitHostPort(opts.HTTPAddress) if err != nil { nsqLog.LogErrorf("failed to parse HTTP address (%s) - %s", opts.HTTPAddress, err) - os.Exit(1) + return nil, err } } statsdHostKey := statsd.HostKey(net.JoinHostPort(opts.BroadcastAddress, port)) @@ -188,7 +234,7 @@ func New(opts *Options) *NSQD { opts.TLSRequired = TLSRequired } - return n + return n, nil } func (n *NSQD) SetReqToEndCB(reqToEndCB ReqToEndFunc) { @@ -488,6 +534,12 @@ func (n *NSQD) Exit() { close(n.exitChan) n.waitGroup.Wait() n.metaStorage.Close() + if n.kvTopicStorage != nil { + n.kvTopicStorage.CloseAll() + } + if n.sharedCfg != nil { + n.sharedCfg.Destroy() + } n.dl.Unlock() nsqLog.Logf("NSQ: exited") @@ -537,6 +589,12 @@ func (n *NSQD) GetTopicWithExt(topicName string, part int, ordered bool) *Topic return n.internalGetTopic(topicName, part, true, ordered, false, 0) } +func (n *NSQD) getKVStorageForTopic(topicName string) engine.KVEngine { + // check if kv topic is enabled for this topic, and check if a isolated storage is needed for this topic + // should not be changed after the data is written + return n.kvTopicStorage +} + func (n *NSQD) internalGetTopic(topicName string, part int, ext bool, ordered bool, disableChannelAutoCreate bool, disabled int32) *Topic { if part > MAX_TOPIC_PARTITION || part < 0 { return nil @@ -571,7 +629,9 @@ func (n *NSQD) internalGetTopic(topicName string, part int, ext bool, ordered bo n.topicMap[topicName] = topics } var t *Topic - t = NewTopicWithExtAndDisableChannelAutoCreate(topicName, part, ext, ordered, disableChannelAutoCreate, n.GetOpts(), disabled, n.metaStorage, n, + t = NewTopicWithExtAndDisableChannelAutoCreate(topicName, part, ext, ordered, + disableChannelAutoCreate, n.GetOpts(), disabled, n.metaStorage, + n.getKVStorageForTopic(topicName), n, n.pubLoopFunc) if t == nil { nsqLog.Errorf("TOPIC(%s): create failed", topicName) @@ -641,7 +701,8 @@ func (n *NSQD) ForceDeleteTopicData(name string, partition int) error { if err != nil { // not exist, create temp for check n.Lock() - topic = NewTopic(name, partition, n.GetOpts(), 1, n.metaStorage, n, + topic = NewTopicForDelete(name, partition, n.GetOpts(), 1, n.metaStorage, + n, n.pubLoopFunc) if topic != nil && !topic.IsOrdered() { // init delayed so we can remove it later @@ -657,12 +718,12 @@ func (n *NSQD) ForceDeleteTopicData(name string, partition int) error { return nil } -func (n *NSQD) CheckMagicCode(name string, partition int, code int64, tryFix bool) (string, error) { +func (n *NSQD) CheckMagicCode(name string, partition int, code int64, isExt bool, tryFix bool) (string, error) { localTopic, err := n.GetExistingTopic(name, partition) if err != nil { // not exist, create temp for check n.Lock() - localTopic = NewTopic(name, partition, n.GetOpts(), 1, n.metaStorage, n, + localTopic = NewTopicWithExt(name, partition, isExt, false, n.GetOpts(), 1, n.metaStorage, n.getKVStorageForTopic(name), n, n.pubLoopFunc) n.Unlock() if localTopic == nil { @@ -738,6 +799,9 @@ func (n *NSQD) flushAll(all bool, flushCnt int) { } if all { n.metaStorage.Sync() + if n.kvTopicStorage != nil { + n.kvTopicStorage.FlushAll() + } } } diff --git a/nsqd/nsqd_test.go b/nsqd/nsqd_test.go index ca96951b..e0601c90 100644 --- a/nsqd/nsqd_test.go +++ b/nsqd/nsqd_test.go @@ -84,6 +84,7 @@ func mustStartNSQD(opts *Options) (*net.TCPAddr, *net.TCPAddr, *NSQD) { opts.TCPAddress = "127.0.0.1:0" opts.HTTPAddress = "127.0.0.1:0" opts.HTTPSAddress = "127.0.0.1:0" + opts.KVEnabled = true if opts.DataPath == "" { tmpDir, err := ioutil.TempDir("", fmt.Sprintf("nsq-test-%d", time.Now().UnixNano())) if err != nil { @@ -91,7 +92,10 @@ func mustStartNSQD(opts *Options) (*net.TCPAddr, *net.TCPAddr, *NSQD) { } opts.DataPath = tmpDir } - nsqd := New(opts) + nsqd, err := New(opts) + if err != nil { + panic(err) + } nsqd.Start() return nil, nil, nsqd } @@ -433,7 +437,9 @@ func TestSkipMetaData(t *testing.T) { func TestSetHealth(t *testing.T) { opts := NewOptions() opts.Logger = newTestLogger(t) - nsqd := New(opts) + opts.KVEnabled = false + nsqd, err := New(opts) + equal(t, err, nil) equal(t, nsqd.GetError(), nil) equal(t, nsqd.IsHealthy(), true) diff --git a/nsqd/options.go b/nsqd/options.go index b0a575f6..d9234622 100644 --- a/nsqd/options.go +++ b/nsqd/options.go @@ -115,6 +115,12 @@ type Options struct { QueueWriteBufferSize int `flag:"queue-write-buffer-size" cfg:"queue_write_buffer_size"` PubQueueSize int `flag:"pub-queue-size" cfg:"pub_queue_size"` SleepMsBetweenLogSyncPull int `flag:"sleepms-between-log-sync-pull" cfg:"sleepms_between_log_sync_pull"` + + // options for kv engine + KVEnabled bool `flag:"kv-enabled" cfg:"kv_enabled"` + KVBlockCache int64 `flag:"kv-block-cache" cfg:"kv_block_cache"` + KVWriteBufferSize int64 `flag:"kv-write-buffer-size" cfg:"kv_write_buffer_size"` + KVMaxWriteBufferNumber int64 `flag:"kv-max-write-buffer-number" cfg:"kv_max_write_buffer_number"` } func NewOptions() *Options { @@ -190,6 +196,7 @@ func NewOptions() *Options { RetentionDays: int32(DEFAULT_RETENTION_DAYS), MaxConnForClient: 500000, + KVEnabled: false, } return opts diff --git a/nsqd/topic.go b/nsqd/topic.go index 4bd60e57..766ad64e 100644 --- a/nsqd/topic.go +++ b/nsqd/topic.go @@ -18,6 +18,7 @@ import ( "github.com/youzan/nsq/internal/protocol" "github.com/youzan/nsq/internal/quantile" "github.com/youzan/nsq/internal/util" + "github.com/youzan/nsq/nsqd/engine" ) const ( @@ -32,6 +33,7 @@ var ( ErrOperationInvalidState = errors.New("the operation is not allowed under current state") ErrMessageInvalidDelayedState = errors.New("the message is invalid for delayed") PubQueue = 500 + errChannelNotExist = errors.New("channel does not exist") ) func writeMessageToBackend(writeExt bool, buf *bytes.Buffer, msg *Message, bq *diskQueueWriter) (BackendOffset, int32, diskQueueEndInfo, error) { @@ -146,6 +148,7 @@ type Topic struct { // the pub data waiting pub ok returned pubWaitingBytes int64 tpLog *levellogger.LevelLogger + kvTopic *KVTopic } func (t *Topic) setExt() { @@ -188,21 +191,24 @@ func GetTopicFullName(topic string, part int) string { return topic + "-" + strconv.Itoa(part) } -func NewTopic(topicName string, part int, opt *Options, +func NewTopicForDelete(topicName string, part int, opt *Options, writeDisabled int32, metaStorage IMetaStorage, notify INsqdNotify, loopFunc func(v *Topic)) *Topic { - return NewTopicWithExt(topicName, part, false, false, opt, writeDisabled, metaStorage, notify, loopFunc) + return NewTopicWithExt(topicName, part, false, false, opt, writeDisabled, metaStorage, nil, notify, loopFunc) } func NewTopicWithExt(topicName string, part int, ext bool, ordered bool, opt *Options, writeDisabled int32, metaStorage IMetaStorage, + kvEng engine.KVEngine, notify INsqdNotify, loopFunc func(v *Topic)) *Topic { - return NewTopicWithExtAndDisableChannelAutoCreate(topicName, part, ext, ordered, false, opt, writeDisabled, metaStorage, notify, loopFunc) + return NewTopicWithExtAndDisableChannelAutoCreate(topicName, part, ext, ordered, false, opt, writeDisabled, metaStorage, kvEng, notify, loopFunc) } // Topic constructor -func NewTopicWithExtAndDisableChannelAutoCreate(topicName string, part int, ext bool, ordered bool, disbaleChannelAutoCreate bool, opt *Options, +func NewTopicWithExtAndDisableChannelAutoCreate(topicName string, part int, ext bool, ordered bool, + disbaleChannelAutoCreate bool, opt *Options, writeDisabled int32, metaStorage IMetaStorage, + kvEng engine.KVEngine, notify INsqdNotify, loopFunc func(v *Topic)) *Topic { if part > MAX_TOPIC_PARTITION { return nil @@ -222,6 +228,7 @@ func NewTopicWithExtAndDisableChannelAutoCreate(topicName string, part int, ext quitChan: make(chan struct{}), pubLoopFunc: loopFunc, metaStorage: metaStorage, + kvTopic: NewKVTopicWithEngine(topicName, part, ext, kvEng), } t.fullName = GetTopicFullName(t.tname, t.partition) t.tpLog = nsqLog.WrappedWithPrefix("["+t.fullName+"]", 0) @@ -275,6 +282,10 @@ func NewTopicWithExtAndDisableChannelAutoCreate(topicName string, part int, ext if err != ErrNeedFixQueueEnd { t.UpdateCommittedOffset(t.backend.GetQueueWriteEnd()) } + if _, err := t.tryFixKVTopic(); err != nil { + t.tpLog.LogErrorf("failed to auto fix kv topic while init: %s", err) + t.SetDataFixState(true) + } err = t.loadMagicCode() if err != nil { t.tpLog.LogErrorf("failed to load magic code: %v", err) @@ -658,11 +669,13 @@ func (t *Topic) UpdateCommittedOffset(offset BackendQueueEnd) { } } -func (t *Topic) GetDiskQueueSnapshot() *DiskQueueSnapshot { +func (t *Topic) GetDiskQueueSnapshot(checkCommit bool) *DiskQueueSnapshot { e := t.backend.GetQueueReadEnd() - commit := t.GetCommitted() - if commit != nil && e.Offset() > commit.Offset() { - e = commit + if checkCommit { + commit := t.GetCommitted() + if commit != nil && e.Offset() > commit.Offset() { + e = commit + } } start := t.backend.GetQueueReadStart() d := NewDiskQueueSnapshot(getBackendName(t.tname, t.partition), t.dataPath, e) @@ -733,6 +746,9 @@ func (t *Topic) SetDynamicInfo(dynamicConf TopicDynamicConf, idGen MsgIDGenerato t.dynamicConf.Ext = dynamicConf.Ext if dynamicConf.Ext { t.setExt() + if t.kvTopic != nil { + t.kvTopic.setExt() + } } t.dynamicConf.DisableChannelAutoCreate = dynamicConf.DisableChannelAutoCreate channelAutoCreateDisabled := t.IsChannelAutoCreateDisabled() @@ -837,7 +853,7 @@ func (t *Topic) getOrCreateChannel(channelName string) (*Channel, bool) { start := t.backend.GetQueueReadStart() channel = NewChannel(t.GetTopicName(), t.GetTopicPart(), t.IsOrdered(), channelName, readEnd, t.option, deleteCallback, t.flushForChannelMoreData, atomic.LoadInt32(&t.writeDisabled), - t.nsqdNotify, ext, start, t.metaStorage, !t.IsDataNeedFix()) + t.nsqdNotify, ext, start, t.metaStorage, t.kvTopic, !t.IsDataNeedFix()) channel.UpdateQueueEnd(readEnd, false) channel.SetDelayedQueue(t.GetDelayedQueue()) @@ -871,7 +887,7 @@ func (t *Topic) CloseExistingChannel(channelName string, deleteData bool) error if numChannels == 0 && t.ephemeral == true { go t.deleter.Do(func() { t.nsqdNotify.NotifyDeleteTopic(t) }) } - return errors.New("channel does not exist") + return errChannelNotExist } t.channelMap[channelName] = nil delete(t.channelMap, channelName) @@ -899,8 +915,11 @@ func (t *Topic) CloseExistingChannel(channelName string, deleteData bool) error // DeleteExistingChannel removes a channel from the topic only if it exists func (t *Topic) DeleteExistingChannel(channelName string) error { err := t.CloseExistingChannel(channelName, true) + if err == errChannelNotExist { + return nil + } if err == nil { - t.SaveChannelMeta() + return t.SaveChannelMeta() } return err } @@ -910,6 +929,9 @@ func (t *Topic) RollbackNoLock(vend BackendOffset, diffCnt uint64) error { t.tpLog.Logf("reset the backend from %v to : %v, %v", old, vend, diffCnt) dend, err := t.backend.RollbackWriteV2(vend, diffCnt) if err == nil { + if t.kvTopic != nil { + t.kvTopic.ResetBackendEnd(vend, dend.TotalMsgCnt()) + } t.UpdateCommittedOffset(&dend) t.updateChannelsEnd(true, true) } @@ -926,6 +948,9 @@ func (t *Topic) ResetBackendEndNoLock(vend BackendOffset, totalCnt int64) error if err != nil { t.tpLog.LogErrorf("reset backend to %v error: %v", vend, err) } else { + if t.kvTopic != nil { + t.kvTopic.ResetBackendEnd(vend, totalCnt) + } t.UpdateCommittedOffset(&dend) t.updateChannelsEnd(true, true) } @@ -1033,6 +1058,17 @@ func (t *Topic) PutRawDataOnReplica(rawData []byte, offset BackendOffset, checkS return &dend, fmt.Errorf("message write size mismatch %v vs %v", checkSize, writeBytes) } atomic.StoreInt32(&t.needFlush, 1) + if t.kvTopic != nil { + kvEnd, kverr := t.kvTopic.PutRawDataOnReplica(rawData, offset, checkSize, msgNum) + if kverr != nil { + t.tpLog.LogWarningf("kv topic write failed: %s, %v", kverr, rawData) + t.SetDataFixState(true) + return &dend, kverr + } else if kvEnd.Offset() != dend.Offset() || kvEnd.TotalMsgCnt() != dend.TotalMsgCnt() { + t.tpLog.LogWarningf("kv topic write end mismatch: %v, %v", kvEnd, dend) + t.SetDataFixState(true) + } + } if atomic.LoadInt32(&t.dynamicConf.AutoCommit) == 1 { t.UpdateCommittedOffset(&dend) } @@ -1148,6 +1184,14 @@ func (t *Topic) put(m *Message, trace bool, checkSize int64) (MessageID, Backend return m.ID, offset, writeBytes, dend, err } + if t.kvTopic != nil { + kvOffset, kvSize, kvEnd, kverr := t.kvTopic.put(m, checkSize) + if kverr != nil { + t.tpLog.LogWarningf("kv topic write failed: %s", kverr) + } else if kvEnd != dend.TotalMsgCnt() || kvOffset != dend.Offset() || kvSize != writeBytes { + t.tpLog.LogWarningf("kv topic write end mismatch: %v-%v vs %v, %v vs %v", kvEnd, kvOffset, dend, kvSize, writeBytes) + } + } if atomic.LoadInt32(&t.dynamicConf.AutoCommit) == 1 { t.UpdateCommittedOffset(&dend) } @@ -1553,6 +1597,9 @@ func (t *Topic) TryCleanOldData(retentionSize int64, noRealClean bool, maxCleanO } t.tpLog.Infof("clean topic data from %v under retention %v, %v", cleanEndInfo, cleanTime, retentionSize) + if !noRealClean && t.kvTopic != nil { + t.kvTopic.TryCleanOldData(retentionSize, cleanEndInfo, maxCleanOffset) + } return t.backend.CleanOldDataByRetention(cleanEndInfo, noRealClean, maxCleanOffset) } @@ -1565,11 +1612,16 @@ func (t *Topic) TryFixQueueEnd(vend BackendOffset, totalCnt int64) error { dend, err := t.backend.TryFixWriteEnd(vend, totalCnt) if err != nil { t.tpLog.LogErrorf("fix backend to %v error: %v", vend, err) - } else { - t.UpdateCommittedOffset(&dend) - t.updateChannelsEnd(true, true) + return err } - return err + // should check if there are kvtopic data missing + t.UpdateCommittedOffset(&dend) + t.updateChannelsEnd(true, true) + _, err = t.tryFixKVTopic() + if err != nil { + return err + } + return nil } func (t *Topic) ResetBackendWithQueueStartNoLock(queueStartOffset int64, queueStartCnt int64) error { @@ -1588,6 +1640,12 @@ func (t *Topic) ResetBackendWithQueueStartNoLock(queueStartOffset int64, queueSt if err != nil { return err } + if t.kvTopic != nil { + err = t.kvTopic.ResetBackendWithQueueStart(queueStartOffset, queueStartCnt) + if err != nil { + return err + } + } newEnd := t.backend.GetQueueReadEnd() t.UpdateCommittedOffset(newEnd) @@ -1647,12 +1705,204 @@ func (t *Topic) UpdateDelayedQueueConsumedState(ts int64, keyList RecentKeyList, } // after crash, some topic meta need to be fixed by manual -func (t *Topic) TryFixData() error { +func (t *Topic) TryFixData(checkCorrupt bool) error { + t.Lock() + defer t.Unlock() t.backend.tryFixData() dq := t.GetDelayedQueue() if dq != nil { dq.backend.tryFixData() } + if checkCorrupt { + err := t.tryFixCorruptData() + if err != nil { + return err + } + } // TODO: fix channel meta + _, err := t.tryFixKVTopic() + return err +} + +func (t *Topic) tryFixCorruptData() error { + // try check the disk queue data by read from start to end, if corrupt is head or tail of queue, + // we can just truncate header or tail. + // if corrupt is in the middle (start, corrupt-pos, end), we should handle like below + // 1.if all consumed is higher than corrupt-pos, we truncate the data between start and oldest consumed + // 2.if some consumed less than corrupt-pos, we truncate the data between start and corrupt-pos, and move consumed pos + // which less than corrupt-pos to the corrupt-pos. + snap := t.GetDiskQueueSnapshot(false) + defer snap.Close() + start := t.backend.GetQueueReadStart() + seekCnt := start.TotalMsgCnt() + lastCorrupt, lastCnt, err := snap.CheckDiskQueueReadToEndOK(int64(start.Offset()), seekCnt, t.getCommittedEnd().Offset()) + if err == nil { + return nil + } + t.tpLog.Warningf("check read failed at: %v, %v, err: %s", lastCorrupt, lastCnt, err) + var oldestPos BackendQueueEnd + t.channelLock.RLock() + for _, ch := range t.channelMap { + pos := ch.GetConfirmed() + if oldestPos == nil { + oldestPos = pos + } else if oldestPos.Offset() > pos.Offset() { + oldestPos = pos + } + } + t.channelLock.RUnlock() + if oldestPos != nil && int64(oldestPos.Offset()) > lastCorrupt { + t.tpLog.Warningf("clean corrupt topic data to %v", oldestPos) + t.backend.CleanOldDataByRetention(oldestPos, false, t.getCommittedEnd().Offset()) + } else { + // check if all the tail is corrupt + snap.ResetToStart() + var fixerr error + for { + fixerr = snap.SkipToNext() + if fixerr == ErrReadEndOfQueue { + break + } + curRead := snap.GetCurrentReadQueueOffset() + if curRead.Offset() > BackendOffset(lastCorrupt) { + break + } + } + if fixerr != nil && fixerr != ErrReadEndOfQueue { + t.tpLog.Warningf("clean corrupt topic data failed, err %s", fixerr) + return fixerr + } + curRead := snap.GetCurrentReadQueueOffset() + if curRead.Offset() <= BackendOffset(lastCorrupt) { + t.tpLog.Warningf("clean corrupt topic data since tail is corrupted, %v", curRead) + // all tail is corrupt, we need truncate the tail + t.ResetBackendEndNoLock(BackendOffset(lastCorrupt), lastCnt) + } else { + t.tpLog.Warningf("clean corrupt topic data since old is corrupted, %v", curRead) + // some middle is corrupt, we just truncate old + newStart, err := t.backend.CleanOldDataByRetention(curRead, false, t.getCommittedEnd().Offset()) + if err != nil { + return err + } + t.tpLog.Warningf("clean corrupt topic data since old is corrupted, %v, %v", curRead, newStart) + t.channelLock.RLock() + for _, ch := range t.channelMap { + pos := ch.GetConfirmed() + if pos.Offset() < newStart.Offset() { + t.tpLog.Infof("channel set new offset %v, old stats: %v", newStart, ch.GetChannelDebugStats()) + ch.SetConsumeOffset(newStart.Offset(), newStart.TotalMsgCnt(), true) + } + } + t.channelLock.RUnlock() + } + } return nil } + +func (t *Topic) CheckDiskQueueReadToEndOK(offset int64, seekCnt int64, endOffset BackendOffset) error { + snap := t.GetDiskQueueSnapshot(false) + defer snap.Close() + _, _, err := snap.CheckDiskQueueReadToEndOK(offset, seekCnt, endOffset) + if err != nil { + t.tpLog.Warningf("check read failed at: %v, err: %s", offset, err) + return err + } + return nil +} + +// should be locked outside +func (t *Topic) tryFixKVTopic() (int64, error) { + // try replay with the disk queue if kv topic data missing some data in end + // try handle some case : + // 1. empty kv data + // 2. queue start not matched + // 3. queue end not matched + if t.kvTopic == nil { + return 0, nil + } + dqEnd := t.backend.GetQueueWriteEnd() + dqStart := t.backend.GetQueueReadStart() + + offset, cnt, err := t.kvTopic.GetTopicMeta() + if err != nil { + return 0, err + } + if cnt > 0 { + if id1, lastCntOffset1, err := t.kvTopic.GetMsgByCnt(cnt - 1); err != nil { + // since the meta is not matched msg, we try fix all index data + t.tpLog.Warningf("kv topic end need fix since last count message not found: %v-%v, %v", offset, cnt, dqEnd) + offset = int64(dqStart.Offset()) + cnt = dqStart.TotalMsgCnt() + // will be reset start below + } else { + id2, offsetCnt, _, lastCntOffset2, err := t.kvTopic.getMsgIDCntTsLessThanOffset(offset) + if err != nil { + t.tpLog.Warningf("kv topic end need fix since last offset message not found: %v-%v, %v", offset, cnt, dqEnd) + offset = int64(dqStart.Offset()) + cnt = dqStart.TotalMsgCnt() + // will be reset start below + } else if id1.ID != id2 || offsetCnt != cnt-1 || lastCntOffset1 != lastCntOffset2 { + t.tpLog.Warningf("kv topic end need fix since last offset message not match: %v-%v, %v vs %v, %v, %v vs %v", offset, cnt, id1, id2, offsetCnt, + lastCntOffset1, lastCntOffset2) + offset = int64(dqStart.Offset()) + cnt = dqStart.TotalMsgCnt() + } + } + } + if dqEnd.TotalMsgCnt() == cnt && dqEnd.Offset() == BackendOffset(offset) { + return 0, nil + } + t.ForceFlushForChannels(true) + snap := t.GetDiskQueueSnapshot(false) + defer snap.Close() + if offset <= int64(dqStart.Offset()) || cnt <= dqStart.TotalMsgCnt() { + t.tpLog.Warningf("kv topic need empty since end is less than queue start: %v-%v, %v", offset, cnt, dqStart) + err = t.kvTopic.ResetBackendWithQueueStart(int64(dqStart.Offset()), dqStart.TotalMsgCnt()) + if err != nil { + return 0, err + } + offset = int64(dqStart.Offset()) + cnt = dqStart.TotalMsgCnt() + if dqStart.Offset() == dqEnd.Offset() { + return 0, nil + } + err = snap.SeekTo(dqStart.Offset(), dqStart.TotalMsgCnt()) + if err != nil { + return 0, err + } + } else { + err = snap.SeekTo(BackendOffset(offset), cnt) + if err != nil { + return 0, err + } + } + t.tpLog.Warningf("kv topic end need fix since not matched: %v-%v, %v", offset, cnt, dqEnd) + fixedCnt := int64(0) + for { + rr := snap.ReadOne() + if rr.Err != nil { + if rr.Err == io.EOF { + break + } + t.tpLog.Warningf("kv topic end fix error: %s, cnt: %v", rr.Err, fixedCnt) + return fixedCnt, rr.Err + } + _, _, _, err = t.kvTopic.putRaw(rr.Data, rr.Offset, int64(rr.MovedSize)) + if err != nil { + t.tpLog.Warningf("kv topic end fix error: %s, %v", err, rr) + return fixedCnt, err + } + fixedCnt++ + } + offset, cnt, err = t.kvTopic.GetTopicMeta() + if err != nil { + t.tpLog.Warningf("kv topic end fix error: %s", err) + return fixedCnt, err + } + t.tpLog.Warningf("kv topic end fixed to: %v-%v, %v", offset, cnt, fixedCnt) + if BackendOffset(offset) != dqEnd.Offset() || cnt != dqEnd.TotalMsgCnt() { + t.tpLog.Warningf("kv topic end not matched after fixed: %v, %v-%v", dqEnd, offset, cnt) + return fixedCnt, errors.New("failed to fix kv topic") + } + return fixedCnt, nil +} diff --git a/nsqd/topic_test.go b/nsqd/topic_test.go index 5416456a..8d7eb71f 100644 --- a/nsqd/topic_test.go +++ b/nsqd/topic_test.go @@ -3,6 +3,7 @@ package nsqd import ( "errors" "os" + "sync" "sync/atomic" //"runtime" @@ -302,7 +303,7 @@ func TestTopicPutChannelWait(t *testing.T) { msg.ID = 0 topic.PutMessage(msg) // wait channel end notify done - time.Sleep(time.Millisecond) + time.Sleep(time.Millisecond * 2000) test.Equal(t, false, channel.IsWaitingMoreData()) test.Equal(t, topic.backend.GetQueueReadEnd(), topic.backend.GetQueueWriteEnd()) test.Equal(t, topic.backend.GetQueueReadEnd(), channel.GetChannelEnd()) @@ -316,13 +317,13 @@ func TestTopicPutChannelWait(t *testing.T) { test.Equal(t, true, channel.IsWaitingMoreData()) msg.ID = 0 topic.PutMessage(msg) - time.Sleep(time.Millisecond) + time.Sleep(time.Millisecond * 2000) test.Equal(t, false, channel.IsWaitingMoreData()) test.Equal(t, topic.backend.GetQueueReadEnd(), topic.backend.GetQueueWriteEnd()) test.Equal(t, topic.backend.GetQueueReadEnd(), channel.GetChannelEnd()) msg.ID = 0 topic.PutMessage(msg) - time.Sleep(time.Millisecond) + time.Sleep(time.Millisecond * 2000) test.NotEqual(t, topic.backend.GetQueueReadEnd(), topic.backend.GetQueueWriteEnd()) test.Equal(t, topic.backend.GetQueueReadEnd(), channel.GetChannelEnd()) } @@ -572,6 +573,7 @@ func TestTopicResetWithQueueStart(t *testing.T) { opts.LogLevel = 3 glog.SetFlags(0, "", "", true, true, 1) glog.StartWorker(time.Second) + SetLogger(opts.Logger) } opts.MaxBytesPerFile = 1024 * 1024 _, _, nsqd := mustStartNSQD(opts) @@ -704,6 +706,317 @@ func TestTopicResetWithQueueStart(t *testing.T) { } } +func TestTopicWriteRollback(t *testing.T) { + opts := NewOptions() + opts.Logger = newTestLogger(t) + opts.MaxBytesPerFile = 1024 * 1024 + _, _, nsqd := mustStartNSQD(opts) + defer os.RemoveAll(opts.DataPath) + defer nsqd.Exit() + + topic := nsqd.GetTopic("test", 0, false) + changeDynamicConfAutCommit(topic.dynamicConf) + + msgNum := 100 + channel := topic.GetChannel("ch") + test.NotNil(t, channel) + msg := NewMessage(0, make([]byte, 1000)) + msg.Timestamp = time.Now().UnixNano() + singleSize := int32(0) + var qend BackendOffset + for i := 0; i < msgNum; i++ { + msg.ID = 0 + _, _, msgSize, dend, err := topic.PutMessage(msg) + test.Nil(t, err) + msg.Timestamp = time.Now().UnixNano() + singleSize = msgSize + qend = dend.Offset() + } + topic.ForceFlush() + + // rollback single + nend := topic.backend.GetQueueWriteEnd() + _, _, err := topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Equal(t, nil, err) + err = topic.RollbackNoLock(qend-BackendOffset(singleSize), 1) + test.Nil(t, err) + if topic.kvTopic != nil { + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Equal(t, ErrMsgNotFoundInIndex, err) + } + nend = topic.backend.GetQueueWriteEnd() + test.Equal(t, qend-BackendOffset(singleSize), nend.Offset()) + test.Equal(t, int64(msgNum-1), nend.TotalMsgCnt()) + if topic.kvTopic != nil { + loffset, lcnt, err := topic.kvTopic.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(nend.Offset()), topic.kvTopic.lastOffset) + test.Equal(t, nend.TotalMsgCnt(), topic.kvTopic.lastCnt) + test.Equal(t, int64(nend.Offset()), loffset) + test.Equal(t, nend.TotalMsgCnt(), lcnt) + } + // rollback batch + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Nil(t, err) + err = topic.RollbackNoLock(qend-BackendOffset(singleSize)*10, 9) + test.Nil(t, err) + if topic.kvTopic != nil { + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Equal(t, ErrMsgNotFoundInIndex, err) + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 9) + test.Equal(t, ErrMsgNotFoundInIndex, err) + } + nend = topic.backend.GetQueueWriteEnd() + test.Equal(t, qend-BackendOffset(singleSize)*10, nend.Offset()) + test.Equal(t, int64(msgNum-10), nend.TotalMsgCnt()) + if topic.kvTopic != nil { + loffset, lcnt, err := topic.kvTopic.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(nend.Offset()), topic.kvTopic.lastOffset) + test.Equal(t, nend.TotalMsgCnt(), topic.kvTopic.lastCnt) + test.Equal(t, int64(nend.Offset()), loffset) + test.Equal(t, nend.TotalMsgCnt(), lcnt) + } +} + +func TestTopicCheckDiskQueueReadToEndOK(t *testing.T) { + opts := NewOptions() + opts.Logger = newTestLogger(t) + opts.MaxBytesPerFile = 1024 * 1024 + _, _, nsqd := mustStartNSQD(opts) + defer os.RemoveAll(opts.DataPath) + defer nsqd.Exit() + + topic := nsqd.GetTopic("test", 0, false) + changeDynamicConfAutCommit(topic.dynamicConf) + + msgNum := 100 + channel := topic.GetChannel("ch") + test.NotNil(t, channel) + msg := NewMessage(0, make([]byte, 100)) + msg.Timestamp = time.Now().UnixNano() + singleSize := BackendOffset(0) + for i := 0; i < msgNum; i++ { + msg.ID = 0 + _, _, msgSize, _, err := topic.PutMessage(msg) + test.Nil(t, err) + msg.Timestamp = time.Now().UnixNano() + singleSize = BackendOffset(msgSize) + } + topic.ForceFlush() + + for i := 0; i < msgNum; i++ { + err := topic.CheckDiskQueueReadToEndOK(int64(i)*int64(singleSize), int64(i), BackendOffset(msgNum)*singleSize) + test.Nil(t, err) + } +} + +func TestTopicFixCorruptData(t *testing.T) { + opts := NewOptions() + opts.Logger = newTestLogger(t) + opts.MaxBytesPerFile = 1024 * 32 + if testing.Verbose() { + opts.LogLevel = 3 + glog.SetFlags(0, "", "", true, true, 1) + glog.StartWorker(time.Second) + SetLogger(opts.Logger) + } + _, _, nsqd := mustStartNSQD(opts) + defer os.RemoveAll(opts.DataPath) + defer nsqd.Exit() + + topic := nsqd.GetTopic("test-corrupt-header", 0, false) + changeDynamicConfAutCommit(topic.dynamicConf) + topic2 := nsqd.GetTopic("test-corrupt-tail", 0, false) + changeDynamicConfAutCommit(topic.dynamicConf) + + msgNum := 100 + channel := topic.GetChannel("ch") + test.NotNil(t, channel) + channel2 := topic2.GetChannel("ch2") + test.NotNil(t, channel2) + channel3 := topic2.GetChannel("ch3") + test.NotNil(t, channel3) + msg := NewMessage(0, make([]byte, 1000)) + msg.Timestamp = time.Now().UnixNano() + singleSize := int32(0) + var qend BackendQueueEnd + for i := 0; i < msgNum; i++ { + msg.ID = 0 + _, _, msgSize, dend, err := topic.PutMessage(msg) + test.Nil(t, err) + msg.Timestamp = time.Now().UnixNano() + singleSize = msgSize + qend = dend + msg.ID = 0 + topic2.PutMessage(msg) + } + topic.ForceFlush() + topic2.ForceFlush() + channel2.skipChannelToEnd() + time.Sleep(time.Second) + test.Equal(t, qend, channel2.GetChannelEnd()) + test.Equal(t, qend, channel3.GetChannelEnd()) + + fs, err := os.OpenFile(topic.backend.fileName(0), os.O_RDWR, 0755) + test.Nil(t, err) + _, err = fs.WriteAt(make([]byte, singleSize/2), 0) + test.Nil(t, err) + fs.Close() + fs, err = os.OpenFile(topic2.backend.fileName(qend.(*diskQueueEndInfo).EndOffset.FileNum), os.O_RDWR, 0755) + test.Nil(t, err) + fs.WriteAt(make([]byte, singleSize), 0) + fs.Close() + + err = topic.tryFixCorruptData() + test.Nil(t, err) + test.Equal(t, qend, topic.GetCommitted()) + time.Sleep(time.Second) + consumed := channel.GetConfirmed() + t.Logf("new consumed: %v", consumed) + test.Equal(t, topic.GetQueueReadStart(), int64(consumed.Offset())) + test.Equal(t, true, consumed.TotalMsgCnt() > 1) + + err = topic2.tryFixCorruptData() + test.Nil(t, err) + t.Logf("new end: %v", topic2.GetCommitted()) + diskEnd := qend.(*diskQueueEndInfo) + test.Equal(t, diskEnd.EndOffset.FileNum, topic2.GetCommitted().(*diskQueueEndInfo).EndOffset.FileNum) + test.Equal(t, int64(0), topic2.GetCommitted().(*diskQueueEndInfo).EndOffset.Pos) + test.Assert(t, diskEnd.Offset() > topic2.GetCommitted().Offset(), "should truncate tail") + test.Assert(t, diskEnd.TotalMsgCnt() > topic2.GetCommitted().TotalMsgCnt(), "should truncate tail") + time.Sleep(time.Second) + consumed = channel2.GetConfirmed() + t.Logf("new consumed: %v", consumed) + test.Equal(t, topic2.getCommittedEnd(), channel2.GetConfirmed()) + test.Equal(t, topic2.getCommittedEnd(), channel2.GetChannelEnd()) + consumed = channel3.GetConfirmed() + t.Logf("new consumed: %v", consumed) + test.Equal(t, topic2.GetQueueReadStart(), channel3.GetConfirmed().TotalMsgCnt()) + test.Equal(t, topic2.getCommittedEnd(), channel3.GetChannelEnd()) +} + +type testEndOffset struct { + offset BackendOffset +} + +func (t *testEndOffset) Offset() BackendOffset { return t.offset } + +func TestTopicFixKV(t *testing.T) { + opts := NewOptions() + opts.Logger = newTestLogger(t) + opts.MaxBytesPerFile = 1024 * 2 + if testing.Verbose() { + opts.LogLevel = 3 + glog.SetFlags(0, "", "", true, true, 1) + glog.StartWorker(time.Second) + SetLogger(opts.Logger) + } + _, _, nsqd := mustStartNSQD(opts) + defer os.RemoveAll(opts.DataPath) + defer nsqd.Exit() + + topic := nsqd.GetTopic("test", 0, false) + changeDynamicConfAutCommit(topic.dynamicConf) + + msgNum := 2000 + channel := topic.GetChannel("ch") + test.NotNil(t, channel) + msg := NewMessage(0, make([]byte, 100)) + msg.Timestamp = time.Now().UnixNano() + singleSize := int32(0) + var qend BackendOffset + var resetStart testEndOffset + for i := 0; i < msgNum; i++ { + msg.ID = 0 + _, _, msgSize, dend, err := topic.PutMessage(msg) + test.Nil(t, err) + msg.Timestamp = time.Now().UnixNano() + singleSize = msgSize + qend = dend.Offset() + if i == msgNum/2 { + resetStart.offset = (dend.Offset()) + } + } + topic.ForceFlush() + + // rollback single + nend := topic.backend.GetQueueWriteEnd() + _, _, err := topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Equal(t, nil, err) + err = topic.kvTopic.ResetBackendEnd(qend-BackendOffset(singleSize), nend.TotalMsgCnt()-1) + test.Nil(t, err) + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Equal(t, ErrMsgNotFoundInIndex, err) + loffset, lcnt, err := topic.kvTopic.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(nend.Offset()-BackendOffset(singleSize)), topic.kvTopic.lastOffset) + test.Equal(t, nend.TotalMsgCnt()-1, topic.kvTopic.lastCnt) + test.Equal(t, int64(nend.Offset()-BackendOffset(singleSize)), loffset) + test.Equal(t, nend.TotalMsgCnt()-1, lcnt) + + fixedCnt, err := topic.tryFixKVTopic() + if err != nil { + t.Log(err.Error()) + } + test.Nil(t, err) + test.Equal(t, int64(1), fixedCnt) + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Equal(t, nil, err) + loffset, lcnt, err = topic.kvTopic.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(nend.Offset()), topic.kvTopic.lastOffset) + test.Equal(t, nend.TotalMsgCnt(), topic.kvTopic.lastCnt) + test.Equal(t, int64(nend.Offset()), loffset) + test.Equal(t, nend.TotalMsgCnt(), lcnt) + + // rollback batch + err = topic.kvTopic.ResetBackendEnd(qend-BackendOffset(singleSize)*10, nend.TotalMsgCnt()-10) + test.Nil(t, err) + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Equal(t, ErrMsgNotFoundInIndex, err) + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 9) + test.Equal(t, ErrMsgNotFoundInIndex, err) + + fixedCnt, err = topic.tryFixKVTopic() + test.Nil(t, err) + test.Equal(t, int64(10), fixedCnt) + loffset, lcnt, err = topic.kvTopic.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(nend.Offset()), topic.kvTopic.lastOffset) + test.Equal(t, nend.TotalMsgCnt(), topic.kvTopic.lastCnt) + test.Equal(t, int64(nend.Offset()), loffset) + test.Equal(t, nend.TotalMsgCnt(), lcnt) + + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Nil(t, err) + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 9) + test.Nil(t, err) + // fix empty kv + topic.kvTopic.Empty() + fixedCnt, err = topic.tryFixKVTopic() + test.Nil(t, err) + test.Equal(t, int64(msgNum), fixedCnt) + // fix which queue start is not 0, and kv is less than start + topic.backend.CleanOldDataByRetention(&resetStart, false, nend.Offset()) + topic.kvTopic.Empty() + fixedCnt, err = topic.tryFixKVTopic() + test.Nil(t, err) + test.Equal(t, int64(msgNum/2+8), fixedCnt) + + loffset, lcnt, err = topic.kvTopic.GetTopicMeta() + test.Nil(t, err) + test.Equal(t, int64(nend.Offset()), topic.kvTopic.lastOffset) + test.Equal(t, nend.TotalMsgCnt(), topic.kvTopic.lastCnt) + test.Equal(t, int64(nend.Offset()), loffset) + test.Equal(t, nend.TotalMsgCnt(), lcnt) + + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 1) + test.Nil(t, err) + _, _, err = topic.kvTopic.GetMsgByCnt(nend.TotalMsgCnt() - 9) + test.Nil(t, err) +} + func TestTopicFixQueueEnd(t *testing.T) { opts := NewOptions() opts.Logger = newTestLogger(t) @@ -711,6 +1024,7 @@ func TestTopicFixQueueEnd(t *testing.T) { opts.LogLevel = 3 glog.SetFlags(0, "", "", true, true, 1) glog.StartWorker(time.Second) + SetLogger(opts.Logger) } opts.MaxBytesPerFile = 1024 * 1024 _, _, nsqd := mustStartNSQD(opts) @@ -785,6 +1099,110 @@ func TestTopicFixQueueEnd(t *testing.T) { test.Equal(t, oldEnd.TotalMsgCnt()+int64(msgNum), newEnd.TotalMsgCnt()) } +func TestTopicWriteConcurrentMulti(t *testing.T) { + opts := NewOptions() + opts.Logger = newTestLogger(t) + opts.MaxBytesPerFile = 1024 * 2 + opts.KVWriteBufferSize = 1024 * 1024 + opts.KVMaxWriteBufferNumber = 2 + if testing.Verbose() { + opts.LogLevel = 3 + glog.SetFlags(0, "", "", true, true, 1) + glog.StartWorker(time.Second) + SetLogger(opts.Logger) + } + _, _, nsqd := mustStartNSQD(opts) + defer os.RemoveAll(opts.DataPath) + defer nsqd.Exit() + + topic := nsqd.GetTopic("test", 0, false) + changeDynamicConfAutCommit(topic.dynamicConf) + topic2 := nsqd.GetTopic("test2", 0, false) + changeDynamicConfAutCommit(topic2.dynamicConf) + + msgNum := 2000 + channel := topic.GetChannel("ch") + test.NotNil(t, channel) + channel2 := topic2.GetChannel("ch") + test.NotNil(t, channel2) + var wg sync.WaitGroup + t.Logf("begin test: %s", time.Now()) + for g := 0; g < 4; g++ { + wg.Add(1) + go func() { + defer wg.Done() + start := time.Now() + for time.Since(start) < time.Second*25 { + topic.kvTopic.kvEng.CompactAllRange() + time.Sleep(time.Second * 5) + } + }() + wg.Add(1) + go func() { + defer wg.Done() + msg := NewMessage(0, make([]byte, 100)) + msg.Timestamp = time.Now().UnixNano() + for i := 0; i < msgNum; i++ { + msg.ID = 0 + msg.Body = []byte("test" + strconv.Itoa(int(i))) + _, _, _, dend, err := topic.PutMessage(msg) + test.Nil(t, err) + if err != nil { + t.Errorf("failed to write : %s", err) + return + } + msg.Timestamp = time.Now().UnixNano() + topic.kvTopic.GetMsgByCnt(int64(i)) + if i%100 == 0 { + topic.ForceFlush() + topic.Lock() + topic.kvTopic.ResetBackendEnd(dend.Offset()/2, dend.TotalMsgCnt()/2) + _, err := topic.tryFixKVTopic() + topic.Unlock() + if err != nil { + t.Errorf("failed to fix kv topic: %s", err) + return + } + } + } + topic.ForceFlush() + }() + wg.Add(1) + go func() { + defer wg.Done() + msg := NewMessage(0, make([]byte, 100)) + msg.Timestamp = time.Now().UnixNano() + for i := 0; i < msgNum; i++ { + msg.ID = 0 + msg.Body = []byte("test2" + strconv.Itoa(int(i))) + _, _, _, dend, err := topic2.PutMessage(msg) + test.Nil(t, err) + if err != nil { + t.Errorf("failed to write : %s", err) + return + } + msg.Timestamp = time.Now().UnixNano() + topic2.kvTopic.GetMsgByCnt(int64(i)) + if i%100 == 0 { + topic2.ForceFlush() + topic2.Lock() + topic2.kvTopic.ResetBackendEnd(dend.Offset()/2, dend.TotalMsgCnt()/2) + _, err := topic2.tryFixKVTopic() + topic2.Unlock() + if err != nil { + t.Errorf("failed to fix kv topic: %s", err) + return + } + } + } + topic2.ForceFlush() + }() + } + + wg.Wait() + t.Logf("end test: %s", time.Now()) +} + func benchmarkTopicPut(b *testing.B, size int, useExt bool) { b.StopTimer() topicName := "bench_topic_put" + strconv.Itoa(b.N) diff --git a/nsqdserver/http.go b/nsqdserver/http.go index 69134ee5..72b60f69 100644 --- a/nsqdserver/http.go +++ b/nsqdserver/http.go @@ -263,7 +263,13 @@ func (s *httpServer) doFixTopicData(w http.ResponseWriter, req *http.Request, ps if err != nil { return nil, http_api.Err{http.StatusInternalServerError, err.Error()} } - localTopic.TryFixData() + reqParams, err := url.ParseQuery(req.URL.RawQuery) + if err != nil { + nsqd.NsqLogger().LogErrorf("failed to parse request params - %s", err) + return nil, http_api.Err{400, "INVALID_REQUEST"} + } + checkCorrupt := reqParams.Get("checkcorrupt") + localTopic.TryFixData(checkCorrupt == "true") if s.ctx.nsqdCoord != nil { err = s.ctx.nsqdCoord.TryFixLocalTopic(localTopic.GetTopicName(), localTopic.GetTopicPart()) @@ -1199,7 +1205,7 @@ func (s *httpServer) doMessageGet(w http.ResponseWriter, req *http.Request, ps h if err != nil { return nil, http_api.Err{404, err.Error()} } - backendReader := t.GetDiskQueueSnapshot() + backendReader := t.GetDiskQueueSnapshot(true) if backendReader == nil { return nil, http_api.Err{500, "Failed to get queue reader"} } @@ -1354,7 +1360,7 @@ func (s *httpServer) doStats(w http.ResponseWriter, req *http.Request, ps httpro leaderOnly, _ = strconv.ParseBool(leaderOnlyStr) jsonFormat := formatString == "json" - filterClients := len(needClients) == 0 + filterClients := needClients != "true" stats := s.ctx.getStats(leaderOnly, topicName, filterClients) health := s.ctx.getHealth() diff --git a/nsqdserver/nsqd_server.go b/nsqdserver/nsqd_server.go index 44911f96..db51c8a7 100644 --- a/nsqdserver/nsqd_server.go +++ b/nsqdserver/nsqd_server.go @@ -116,7 +116,11 @@ func NewNsqdServer(opts *nsqd.Options) (*nsqd.NSQD, *NsqdServer, error) { } nsqd.NsqLogger().Logf("current max conn for client is: %v", opts.MaxConnForClient) - nsqdInstance := nsqd.New(opts) + nsqdInstance, err := nsqd.New(opts) + if err != nil { + nsqd.NsqLogger().LogErrorf("FATAL: failed to init nsqd- %s", err) + return nil, nil, err + } s := &NsqdServer{} ctx := &context{} diff --git a/nsqdserver/nsqd_server_test.go b/nsqdserver/nsqd_server_test.go index 7dfcb4a2..b73d5980 100644 --- a/nsqdserver/nsqd_server_test.go +++ b/nsqdserver/nsqd_server_test.go @@ -57,6 +57,7 @@ func mustStartNSQD(opts *nsqdNs.Options) (*net.TCPAddr, *net.TCPAddr, *nsqdNs.NS opts.TCPAddress = "127.0.0.1:0" opts.HTTPAddress = "127.0.0.1:0" opts.HTTPSAddress = "127.0.0.1:0" + opts.KVEnabled = true if opts.DataPath == "" { tmpDir, err := ioutil.TempDir("", fmt.Sprintf("nsq-test-%d", time.Now().UnixNano())) if err != nil { diff --git a/nsqdserver/protocol_v2_test.go b/nsqdserver/protocol_v2_test.go index abb8c65d..d724270d 100644 --- a/nsqdserver/protocol_v2_test.go +++ b/nsqdserver/protocol_v2_test.go @@ -10,6 +10,7 @@ import ( "errors" "fmt" "io" + "io/ioutil" "math" "math/rand" "net" @@ -1911,24 +1912,24 @@ func TestConsumeMultiTagMessages(t *testing.T) { var wg sync.WaitGroup wg.Add(1) go func() { + defer wg.Done() msgOut := recvNextMsgAndCheckWithCloseChan(t, conn, len(msg.Body), msg.TraceID, true, true, closeChan) test.Nil(t, msgOut) t.Logf("subscrieb without tag stops.") - wg.Done() }() var wgTag sync.WaitGroup wgTag.Add(1) go func() { + defer wgTag.Done() msgOut1 := recvNextMsgAndCheckExt(t, conn1, len(msg.Body), msg.TraceID, true, true) test.NotNil(t, msgOut1) - wgTag.Done() }() wgTag.Add(1) go func() { + defer wgTag.Done() msgOut2 := recvNextMsgAndCheckExt(t, conn2, len(msg.Body), msg.TraceID, true, true) test.NotNil(t, msgOut2) - wgTag.Done() }() wgTag.Wait() @@ -5650,6 +5651,7 @@ func TestSubOrderedWithFilter(t *testing.T) { func TestSubWithLargeReady(t *testing.T) { opts := nsqdNs.NewOptions() opts.Logger = newTestLogger(t) + opts.MaxRdyCount = 250 if testing.Verbose() { opts.LogLevel = 2 nsqdNs.SetLogger(opts.Logger) @@ -5668,7 +5670,7 @@ func TestSubWithLargeReady(t *testing.T) { msg := nsqdNs.NewMessage(0, []byte("test body")) topic.PutMessage(msg) - for i := 0; i < 100000; i++ { + for i := 0; i < 1000; i++ { topic.PutMessage(nsqdNs.NewMessage(0, []byte("test body"))) } @@ -5677,13 +5679,13 @@ func TestSubWithLargeReady(t *testing.T) { defer conn.Close() - _, err = nsq.Ready(2500).WriteTo(conn) + _, err = nsq.Ready(250).WriteTo(conn) test.Equal(t, err, nil) _, err = nsq.Ready(int(opts.MaxRdyCount)).WriteTo(conn) test.Equal(t, err, nil) - for i := 0; i < 100000; i++ { + for i := 0; i < 1000; i++ { msgOut := recvNextMsgAndCheckClientMsg(t, conn, len(msg.Body), 0, false) _, err = nsq.Finish(msgOut.ID).WriteTo(conn) if err != nil { @@ -7372,11 +7374,22 @@ func testIOLoopReturnsClientErr(t *testing.T, fakeConn test.FakeNetConn) { opts := nsqdNs.NewOptions() opts.Logger = newTestLogger(t) + opts.KVEnabled = false + if opts.DataPath == "" { + tmpDir, err := ioutil.TempDir("", fmt.Sprintf("nsq-test-%d", time.Now().UnixNano())) + if err != nil { + panic(err) + } + opts.DataPath = tmpDir + defer os.RemoveAll(tmpDir) + } - prot := &protocolV2{ctx: &context{nsqd: nsqdNs.New(opts)}} + nd, err := nsqdNs.New(opts) + test.Nil(t, err) + prot := &protocolV2{ctx: &context{nsqd: nd}} defer prot.ctx.nsqd.Exit() - err := prot.IOLoop(fakeConn) + err = prot.IOLoop(fakeConn) test.NotNil(t, err) test.Equal(t, strings.HasPrefix(err.Error(), "E_INVALID "), true)