Update docs: master::syncMsgProducer

Signed-off-by: GuoRentong <rentong.guo@zilliz.com>
This commit is contained in:
GuoRentong 2020-11-21 11:39:23 +08:00 committed by yefu.chen
parent aabe9afd75
commit 8b88a9c306
19 changed files with 471 additions and 129 deletions

View File

@ -171,6 +171,10 @@ func (dispatcher *MarshalDispatcher) addDefaultMsgTemplates()
func NewUnmarshalDispatcher() *UnmarshalDispatcher
```
#### A.4 Time Ticked Flow Graph
###### A.4.1 Flow Graph States
@ -274,9 +278,90 @@ func NewIdAllocator(ctx context.Context) *IdAllocator
#### A.6 KV
###### A.6.1 KV Base
#### A.6 Timestamp Allocator
###### A.6.1 Timestamp
Let's take a brief review of Hybrid Logical Clock (HLC). HLC uses 64bits timestamps which are composed of a 46-bits physical component (thought of as and always close to local wall time) and a 18-bits logical component (used to distinguish between events with the same physical component).
<img src="/Users/grt/Project/grt/milvus-distributed/docs/developer_guides/figs/hlc.png" width=400>
HLC's logical part is advanced on each request. The phsical part can be increased in two cases:
A. when the local wall time is greater than HLC's physical part,
B. or the logical part overflows.
In either cases, the physical part will be updated, and the logical part will be set to 0.
Keep the physical part close to local wall time may face non-monotonic problems such as updates to POSIX time that could turn time backward. HLC avoids such problems, since if 'local wall time < HLC's physical part' holds, only case B is satisfied, thus montonicity is guaranteed.
Milvus does not support transaction, but it should gurantee the deterministic execution of the multi-way WAL. The timestamp attached to each request should
- have its physical part close to wall time (has an acceptable bounded error, a.k.a. uncertainty interval in transaction senarios),
- and be globally unique.
HLC leverages on physical clocks at nodes that are synchronized using the NTP. NTP usually maintain time to within tens of milliseconds over local networks in datacenter. Asymmetric routes and network congestion occasionally cause errors of hundreds of milliseconds. Both the normal time error and the spike are acceptable for Milvus use cases.
The interface of Timestamp is as follows.
```
type timestamp struct {
physical uint64 // 18-63 bits
logical uint64 // 0-17 bits
}
type Timestamp uint64
```
###### A.6.2 Timestamp Oracle
```go
type timestampOracle struct {
client *etcd.Client // client of a reliable meta service, i.e. etcd client
rootPath string // this timestampOracle's working root path on the reliable kv service
saveInterval uint64
lastSavedTime uint64
tso Timestamp // monotonically increasing timestamp
}
func (tso *timestampOracle) GetTimestamp(count uint32) ([]Timestamp, error)
func (tso *timestampOracle) saveTimestamp() error
func (tso *timestampOracle) loadTimestamp() error
```
###### A.6.3 Timestamp Allocator
```go
type TimestampAllocator struct {}
func (allocator *TimestampAllocator) Start() error
func (allocator *TimestampAllocator) Close() error
func (allocator *TimestampAllocator) Alloc(count uint32) ([]Timestamp, error)
func NewTimestampAllocator() *TimestampAllocator
```
* Batch Allocation of Timestamps
* Expiration of Timestamps
#### A.7 KV
###### A.7.1 KV Base
```go
type KVBase interface {
@ -301,7 +386,7 @@ type KVBase interface {
###### A.6.2 Etcd KV
###### A.7.2 Etcd KV
```go
type EtcdKV struct {

View File

@ -0,0 +1,2 @@
## Appendix D. Error Code

View File

@ -0,0 +1,2 @@
## Appendix E. Statistics

View File

@ -66,3 +66,8 @@ Each of the WAL is attached with a timestamp, which is the time when the log is
For better throughput, Milvus allows asynchronous state synchronization between WAL and index/binlog/table. Whenever the data is not fresh enough to satisfiy a query, the query will be suspended until the data is up-to-date, or timeout will be returned.
#### 1.5 Stream and Time
In order to boost throughput, we model Milvus as a stream-driven system.

View File

@ -1,95 +0,0 @@
## 4. Time
#### 4.1 Overview
In order to boost throughput, we model Milvus as a stream processing system.
#### 4.5 T_safe
#### 4.1 Timestamp
Let's take a brief review of Hybrid Logical Clock (HLC). HLC uses 64bits timestamps which are composed of a 46-bits physical component (thought of as and always close to local wall time) and a 18-bits logical component (used to distinguish between events with the same physical component).
<img src="/Users/grt/Project/grt/milvus-distributed/docs/developer_guides/figs/hlc.png" width=400>
HLC's logical part is advanced on each request. The phsical part can be increased in two cases:
A. when the local wall time is greater than HLC's physical part,
B. or the logical part overflows.
In either cases, the physical part will be updated, and the logical part will be set to 0.
Keep the physical part close to local wall time may face non-monotonic problems such as updates to POSIX time that could turn time backward. HLC avoids such problems, since if 'local wall time < HLC's physical part' holds, only case B is satisfied, thus montonicity is guaranteed.
Milvus does not support transaction, but it should gurantee the deterministic execution of the multi-way WAL. The timestamp attached to each request should
- have its physical part close to wall time (has an acceptable bounded error, a.k.a. uncertainty interval in transaction senarios),
- and be globally unique.
HLC leverages on physical clocks at nodes that are synchronized using the NTP. NTP usually maintain time to within tens of milliseconds over local networks in datacenter. Asymmetric routes and network congestion occasionally cause errors of hundreds of milliseconds. Both the normal time error and the spike are acceptable for Milvus use cases.
The interface of Timestamp is as follows.
```
type timestamp struct {
physical uint64 // 18-63 bits
logical uint64 // 0-17 bits
}
type Timestamp uint64
```
#### 4.2 Timestamp Oracle
```go
type timestampOracle struct {
client *etcd.Client // client of a reliable meta service, i.e. etcd client
rootPath string // this timestampOracle's working root path on the reliable kv service
saveInterval uint64
lastSavedTime uint64
tso Timestamp // monotonically increasing timestamp
}
func (tso *timestampOracle) GetTimestamp(count uint32) ([]Timestamp, error)
func (tso *timestampOracle) saveTimestamp() error
func (tso *timestampOracle) loadTimestamp() error
```
#### 4.2 Timestamp Allocator
```go
type TimestampAllocator struct {}
func (allocator *TimestampAllocator) Start() error
func (allocator *TimestampAllocator) Close() error
func (allocator *TimestampAllocator) Alloc(count uint32) ([]Timestamp, error)
func NewTimestampAllocator() *TimestampAllocator
```
###### 4.2.1 Batch Allocation of Timestamps
###### 4.2.2 Expiration of Timestamps

View File

@ -10,6 +10,8 @@
TODO: a formal acknowledgement.
main content: Rentong Guo, Qingxiang Chen (appendix b)
figures: Xuan Yang, Zhenshan Cao
design suggestions: Zhenshan Cao, Xi Ge, Yefu Chen, Guilin Gou, Yihao Dai, Jiquan Long, Xiaomeng Yi, Peng Xu, Hai Jin, Xiangzhou Guo

Binary file not shown.

Before

Width:  |  Height:  |  Size: 300 KiB

After

Width:  |  Height:  |  Size: 361 KiB

View File

@ -59,21 +59,21 @@ class Schema {
}
const std::vector<int>&
get_sizeof_infos() {
get_sizeof_infos() const {
return sizeof_infos_;
}
std::optional<int>
get_offset(const std::string& field_name) {
get_offset(const std::string& field_name) const {
if (!offsets_.count(field_name)) {
return std::nullopt;
} else {
return offsets_[field_name];
return offsets_.at(field_name);
}
}
const std::vector<FieldMeta>&
get_fields() {
get_fields() const {
return fields_;
}

View File

@ -67,7 +67,7 @@ struct TermExpr : Expr {
struct RangeExpr : Expr {
FieldId field_id_;
DataType data_type_ = DataType::NONE;
enum class OpType { LessThan, LessEqual, GreaterThan, GreaterEqual, Equal, NotEqual };
enum class OpType { GreaterThan = 0, GreaterEqual = 1, LessThan = 2, LessEqual = 3, Equal, NotEqual };
static const std::map<std::string, OpType> mapping_; // op_name -> op
// std::vector<std::tuple<OpType, std::any>> conditions_;

View File

@ -1,8 +1,9 @@
#pragma once
// Generated File
// DO NOT EDIT
#include "segcore/SegmentNaive.h"
#include "segcore/SegmentSmallIndex.h"
#include <optional>
#include "query/ExprImpl.h"
#include "ExprVisitor.h"
namespace milvus::query {
@ -21,8 +22,8 @@ class ExecExprVisitor : ExprVisitor {
visit(RangeExpr& expr) override;
public:
using RetType = faiss::ConcurrentBitsetPtr;
explicit ExecExprVisitor(segcore::SegmentNaive& segment) : segment_(segment) {
using RetType = std::vector<std::vector<bool>>;
explicit ExecExprVisitor(segcore::SegmentSmallIndex& segment) : segment_(segment) {
}
RetType
call_child(Expr& expr) {
@ -34,8 +35,17 @@ class ExecExprVisitor : ExprVisitor {
return std::move(ret.value());
}
public:
template <typename Tp, typename Func>
auto
ExecRangeVisitorImpl(RangeExprImpl<Tp>& expr_scp, Func func) -> RetType;
template <typename T>
auto
ExecRangeVisitorDispatcher(RangeExpr& expr_raw) -> RetType;
private:
segcore::SegmentNaive& segment_;
segcore::SegmentSmallIndex& segment_;
std::optional<RetType> ret_;
};
} // namespace milvus::query

View File

@ -1,5 +1,6 @@
#include "segcore/SegmentNaive.h"
#include "segcore/SegmentSmallIndex.h"
#include <optional>
#include "query/ExprImpl.h"
#include "query/generated/ExecExprVisitor.h"
namespace milvus::query {
@ -9,8 +10,8 @@ namespace milvus::query {
namespace impl {
class ExecExprVisitor : ExprVisitor {
public:
using RetType = faiss::ConcurrentBitsetPtr;
explicit ExecExprVisitor(segcore::SegmentNaive& segment) : segment_(segment) {
using RetType = std::vector<std::vector<bool>>;
explicit ExecExprVisitor(segcore::SegmentSmallIndex& segment) : segment_(segment) {
}
RetType
call_child(Expr& expr) {
@ -22,8 +23,17 @@ class ExecExprVisitor : ExprVisitor {
return std::move(ret.value());
}
public:
template <typename Tp, typename Func>
auto
ExecRangeVisitorImpl(RangeExprImpl<Tp>& expr_scp, Func func) -> RetType;
template <typename T>
auto
ExecRangeVisitorDispatcher(RangeExpr& expr_raw) -> RetType;
private:
segcore::SegmentNaive& segment_;
segcore::SegmentSmallIndex& segment_;
std::optional<RetType> ret_;
};
} // namespace impl
@ -44,8 +54,137 @@ ExecExprVisitor::visit(TermExpr& expr) {
PanicInfo("unimplemented");
}
template <typename T, typename Func>
auto
ExecExprVisitor::ExecRangeVisitorImpl(RangeExprImpl<T>& expr, Func func) -> RetType {
auto& records = segment_.get_insert_record();
auto data_type = expr.data_type_;
auto& schema = segment_.get_schema();
auto field_offset_opt = schema.get_offset(expr.field_id_);
Assert(field_offset_opt);
auto field_offset = field_offset_opt.value();
auto& field_meta = schema[field_offset];
auto vec_ptr = records.get_scalar_entity<T>(field_offset);
auto& vec = *vec_ptr;
std::vector<std::vector<bool>> results(vec.chunk_size());
for (auto chunk_id = 0; chunk_id < vec.chunk_size(); ++chunk_id) {
auto& result = results[chunk_id];
result.resize(segcore::DefaultElementPerChunk);
auto chunk = vec.get_chunk(chunk_id);
const T* data = chunk.data();
for (int index = 0; index < segcore::DefaultElementPerChunk; ++index) {
result[index] = func(data[index]);
}
}
return results;
}
#pragma clang diagnostic push
#pragma ide diagnostic ignored "Simplify"
template <typename T>
auto
ExecExprVisitor::ExecRangeVisitorDispatcher(RangeExpr& expr_raw) -> RetType {
auto& expr = static_cast<RangeExprImpl<T>&>(expr_raw);
auto conditions = expr.conditions_;
std::sort(conditions.begin(), conditions.end());
using OpType = RangeExpr::OpType;
if (conditions.size() == 1) {
auto cond = conditions[0];
// auto [op, val] = cond; // strange bug on capture
auto op = std::get<0>(cond);
auto val = std::get<1>(cond);
switch (op) {
case OpType::Equal: {
return ExecRangeVisitorImpl(expr, [val](T x) { return !(x == val); });
}
case OpType::NotEqual: {
return ExecRangeVisitorImpl(expr, [val](T x) { return !(x != val); });
}
case OpType::GreaterEqual: {
return ExecRangeVisitorImpl(expr, [val](T x) { return !(x >= val); });
}
case OpType::GreaterThan: {
return ExecRangeVisitorImpl(expr, [val](T x) { return !(x > val); });
}
case OpType::LessEqual: {
return ExecRangeVisitorImpl(expr, [val](T x) { return !(x <= val); });
}
case OpType::LessThan: {
return ExecRangeVisitorImpl(expr, [val](T x) { return !(x < val); });
}
default: {
PanicInfo("unsupported range node");
}
}
} else if (conditions.size() == 2) {
OpType op1, op2;
T val1, val2;
std::tie(op1, val1) = conditions[0];
std::tie(op2, val2) = conditions[1];
Assert(val1 <= val2);
auto ops = std::make_tuple(op1, op2);
if (false) {
} else if (ops == std::make_tuple(OpType::GreaterThan, OpType::LessThan)) {
return ExecRangeVisitorImpl(expr, [val1, val2](T x) { return !(val1 < x && x < val2); });
} else if (ops == std::make_tuple(OpType::GreaterThan, OpType::LessEqual)) {
return ExecRangeVisitorImpl(expr, [val1, val2](T x) { return !(val1 < x && x <= val2); });
} else if (ops == std::make_tuple(OpType::GreaterEqual, OpType::LessThan)) {
return ExecRangeVisitorImpl(expr, [val1, val2](T x) { return !(val1 <= x && x < val2); });
} else if (ops == std::make_tuple(OpType::GreaterEqual, OpType::LessEqual)) {
return ExecRangeVisitorImpl(expr, [val1, val2](T x) { return !(val1 <= x && x <= val2); });
} else {
PanicInfo("unsupported range node");
}
} else {
PanicInfo("unsupported range node");
}
}
#pragma clang diagnostic pop
void
ExecExprVisitor::visit(RangeExpr& expr) {
auto& field_meta = segment_.get_schema()[expr.field_id_];
Assert(expr.data_type_ == field_meta.get_data_type());
RetType ret;
switch (expr.data_type_) {
// case DataType::BOOL: {
// ret = ExecRangeVisitorDispatcher<bool>(expr);
// break;
//}
case DataType::BOOL:
case DataType::INT8: {
ret = ExecRangeVisitorDispatcher<int8_t>(expr);
break;
}
case DataType::INT16: {
ret = ExecRangeVisitorDispatcher<int16_t>(expr);
break;
}
case DataType::INT32: {
ret = ExecRangeVisitorDispatcher<int32_t>(expr);
break;
}
case DataType::INT64: {
ret = ExecRangeVisitorDispatcher<int64_t>(expr);
break;
}
case DataType::FLOAT: {
ret = ExecRangeVisitorDispatcher<float>(expr);
break;
}
case DataType::DOUBLE: {
ret = ExecRangeVisitorDispatcher<double>(expr);
break;
}
default:
PanicInfo("unsupported");
}
ret_ = std::move(ret);
}
} // namespace milvus::query

View File

@ -12,11 +12,36 @@ struct InsertRecord {
std::vector<std::shared_ptr<VectorBase>> entity_vec_;
InsertRecord(const Schema& schema);
template <typename Type>
auto
get_scalar_entity(int offset) const {
auto ptr = std::dynamic_pointer_cast<const ConcurrentVector<Type, true>>(entity_vec_[offset]);
Assert(ptr);
return ptr;
}
template <typename Type>
auto
get_vec_entity(int offset) const {
auto ptr = std::dynamic_pointer_cast<const ConcurrentVector<Type>>(entity_vec_[offset]);
Assert(ptr);
return ptr;
}
template <typename Type>
auto
get_scalar_entity(int offset) {
auto ptr = std::dynamic_pointer_cast<ConcurrentVector<Type, true>>(entity_vec_[offset]);
Assert(ptr);
return ptr;
}
template <typename Type>
auto
get_vec_entity(int offset) {
return std::static_pointer_cast<ConcurrentVector<Type>>(entity_vec_[offset]);
auto ptr = std::dynamic_pointer_cast<ConcurrentVector<Type>>(entity_vec_[offset]);
Assert(ptr);
return ptr;
}
};
} // namespace milvus::segcore

View File

@ -14,6 +14,7 @@
#include "query/PlanNode.h"
#include "query/PlanImpl.h"
#include "segcore/Reduce.h"
#include "utils/tools.h"
namespace milvus::segcore {
@ -432,13 +433,6 @@ SegmentSmallIndex::BuildIndex(IndexMetaPtr remote_index_meta) {
#endif
}
static uint64_t
upper_align(int64_t value, int64_t align) {
Assert(align > 0);
Assert((align & (align - 1)) == 0);
auto groups = (value + align - 1) / align;
return groups * align;
}
int64_t
SegmentSmallIndex::GetMemoryUsageInBytes() {

View File

@ -107,9 +107,17 @@ class SegmentSmallIndex : public SegmentBase {
GetMemoryUsageInBytes() override;
public:
void
get_insert_record();
const InsertRecord&
get_insert_record() const {
return record_;
}
const Schema&
get_schema() const {
return *schema_;
}
public:
ssize_t
get_row_count() const override {
return record_.ack_responder_.GetAck();

View File

@ -0,0 +1,19 @@
#pragma once
#include <cstdint>
#include "EasyAssert.h"
namespace milvus {
inline int64_t
upper_align(int64_t value, int64_t align) {
Assert(align > 0);
auto groups = (value + align - 1) / align;
return groups * align;
}
inline int64_t
upper_div(int64_t value, int64_t align) {
Assert(align > 0);
auto groups = (value + align - 1) / align;
return groups;
}
}

View File

@ -6,7 +6,11 @@
#include "query/generated/PlanNodeVisitor.h"
#include "test_utils/DataGen.h"
#include "query/generated/ShowPlanNodeVisitor.h"
#include "query/generated/ExecExprVisitor.h"
#include "query/Plan.h"
#include "utils/tools.h"
#include <regex>
#include "segcore/SegmentSmallIndex.h"
using namespace milvus;
TEST(Expr, Naive) {
@ -112,3 +116,84 @@ TEST(Expr, ShowExecutor) {
dup["data"] = "...collased...";
std::cout << dup.dump(4);
}
TEST(Expr, TestRange) {
using namespace milvus::query;
using namespace milvus::segcore;
std::vector<std::tuple<std::string, std::function<bool(int)>>> testcases = {
{R"("GT": 2000, "LT": 3000)", [](int v) { return 2000 < v && v < 3000; }},
{R"("GE": 2000, "LT": 3000)", [](int v) { return 2000 <= v && v < 3000; }},
{R"("GT": 2000, "LE": 3000)", [](int v) { return 2000 < v && v <= 3000; }},
{R"("GE": 2000, "LE": 3000)", [](int v) { return 2000 <= v && v <= 3000; }},
{R"("GE": 2000)", [](int v) { return v >= 2000; }},
{R"("GT": 2000)", [](int v) { return v > 2000; }},
{R"("LE": 2000)", [](int v) { return v <= 2000; }},
{R"("LT": 2000)", [](int v) { return v < 2000; }},
{R"("EQ": 2000)", [](int v) { return v == 2000; }},
{R"("NE": 2000)", [](int v) { return v != 2000; }},
};
std::string dsl_string_tmp = R"(
{
"bool": {
"must": [
{
"range": {
"age": {
@@@@
}
}
},
{
"vector": {
"fakevec": {
"metric_type": "L2",
"params": {
"nprobe": 10
},
"query": "$0",
"topk": 10
}
}
}
]
}
})";
auto schema = std::make_shared<Schema>();
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16);
schema->AddField("age", DataType::INT32);
auto seg = CreateSegment(schema);
int N = 10000;
std::vector<int> age_col;
int num_iters = 100;
for (int iter = 0; iter < num_iters; ++iter) {
auto raw_data = DataGen(schema, N, iter);
auto new_age_col = raw_data.get_col<int>(1);
age_col.insert(age_col.end(), new_age_col.begin(), new_age_col.end());
seg->PreInsert(N);
seg->Insert(iter * N, N, raw_data.row_ids_.data(), raw_data.timestamps_.data(), raw_data.raw_);
}
auto seg_promote = dynamic_cast<SegmentSmallIndex*>(seg.get());
ExecExprVisitor visitor(*seg_promote);
for (auto [clause, ref_func]: testcases) {
auto loc = dsl_string_tmp.find("@@@@");
auto dsl_string = dsl_string_tmp;
dsl_string.replace(loc, 4, clause);
auto plan = CreatePlan(*schema, dsl_string);
auto final = visitor.call_child(*plan->plan_node_->predicate_.value());
EXPECT_EQ(final.size(), upper_div(N * num_iters, DefaultElementPerChunk));
for (int i = 0; i < N * num_iters; ++i) {
auto vec_id = i / DefaultElementPerChunk;
auto offset = i % DefaultElementPerChunk;
auto ans = final[vec_id][offset];
auto val = age_col[i];
auto ref = !ref_func(val);
ASSERT_EQ(ans, ref) << clause << "@" << i
<< "!!" << val;
}
}
}

View File

@ -3,13 +3,15 @@
#include <random>
#include <memory>
#include <cstring>
#include "segcore/SegmentBase.h"
namespace milvus::segcore {
struct GeneratedData {
std::vector<char> rows_;
std::vector<std::vector<char>> cols_;
void
generate_rows(int N, SchemaPtr schema);
std::vector<idx_t> row_ids_;
std::vector<Timestamp> timestamps_;
RowBasedRawData raw_;
template <typename T>
auto
get_col(int index) {
@ -18,8 +20,14 @@ struct GeneratedData {
memcpy(ret.data(), target.data(), target.size());
return ret;
}
};
private:
GeneratedData() = default;
friend GeneratedData
DataGen(SchemaPtr schema, int64_t N, uint64_t seed);
void
generate_rows(int N, SchemaPtr schema);
};
inline void
GeneratedData::generate_rows(int N, SchemaPtr schema) {
std::vector<int> offset_infos(schema->size() + 1, 0);
@ -42,10 +50,10 @@ GeneratedData::generate_rows(int N, SchemaPtr schema) {
}
inline GeneratedData
DataGen(SchemaPtr schema, int64_t N) {
DataGen(SchemaPtr schema, int64_t N, uint64_t seed = 42) {
using std::vector;
std::vector<vector<char>> cols;
std::default_random_engine er(42);
std::default_random_engine er(seed);
std::normal_distribution<> distr(0, 1);
int offset = 0;
@ -70,10 +78,16 @@ DataGen(SchemaPtr schema, int64_t N) {
}
case engine::DataType::INT32: {
vector<int> data(N);
int count = 0;
for (auto& x : data) {
x = count + offset * N;
++count;
x = er() % (2 * N);
}
insert_cols(data);
break;
}
case engine::DataType::FLOAT: {
vector<float> data(N);
for (auto& x : data) {
x = distr(er);
}
insert_cols(data);
break;
@ -87,6 +101,13 @@ DataGen(SchemaPtr schema, int64_t N) {
GeneratedData res;
res.cols_ = std::move(cols);
res.generate_rows(N, schema);
for (int i = 0; i < N; ++i) {
res.row_ids_.push_back(i);
res.timestamps_.push_back(i);
}
res.raw_.raw_data = res.rows_.data();
res.raw_.sizeof_per_row = schema->get_total_sizeof();
res.raw_.count = N;
return std::move(res);
}

View File

@ -372,6 +372,46 @@ func (ms *PulsarTtMsgStream) findTimeTick(channelIndex int,
}
}
//TODO test InMemMsgStream
/*
type InMemMsgStream struct {
buffer chan *MsgPack
}
func (ms *InMemMsgStream) Start() {}
func (ms *InMemMsgStream) Close() {}
func (ms *InMemMsgStream) ProduceOne(msg TsMsg) error {
msgPack := MsgPack{}
msgPack.BeginTs = msg.BeginTs()
msgPack.EndTs = msg.EndTs()
msgPack.Msgs = append(msgPack.Msgs, msg)
buffer <- &msgPack
return nil
}
func (ms *InMemMsgStream) Produce(msgPack *MsgPack) error {
buffer <- msgPack
return nil
}
func (ms *InMemMsgStream) Broadcast(msgPack *MsgPack) error {
return ms.Produce(msgPack)
}
func (ms *InMemMsgStream) Consume() *MsgPack {
select {
case msgPack := <-ms.buffer:
return msgPack
}
}
func (ms *InMemMsgStream) Chan() <- chan *MsgPack {
return buffer
}
*/
func checkTimeTickMsg(msg map[int]Timestamp) (Timestamp, bool) {
checkMap := make(map[Timestamp]int)
for _, v := range msg {