提交 94c3fc93 编写于 作者: S sunby 提交者: yefu.chen

Add unit tests in dataservice

Signed-off-by: Nsunby <bingyi.sun@zilliz.com>
上级 a5409770
package dataservice
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/types"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
)
type ddHandler struct {
meta *meta
segmentAllocator segmentAllocatorInterface
masterClient types.MasterService
}
func newDDHandler(meta *meta, allocator segmentAllocatorInterface, client types.MasterService) *ddHandler {
return &ddHandler{
meta: meta,
segmentAllocator: allocator,
masterClient: client,
}
}
func (handler *ddHandler) HandleDDMsg(ctx context.Context, msg msgstream.TsMsg) error {
switch msg.Type() {
case commonpb.MsgType_CreateCollection:
realMsg := msg.(*msgstream.CreateCollectionMsg)
return handler.handleCreateCollection(realMsg)
case commonpb.MsgType_DropCollection:
realMsg := msg.(*msgstream.DropCollectionMsg)
return handler.handleDropCollection(ctx, realMsg)
case commonpb.MsgType_CreatePartition:
realMsg := msg.(*msgstream.CreatePartitionMsg)
return handler.handleCreatePartition(realMsg)
case commonpb.MsgType_DropPartition:
realMsg := msg.(*msgstream.DropPartitionMsg)
return handler.handleDropPartition(ctx, realMsg)
default:
return nil
}
}
func (handler *ddHandler) handleCreateCollection(msg *msgstream.CreateCollectionMsg) error {
schema := &schemapb.CollectionSchema{}
if err := proto.Unmarshal(msg.Schema, schema); err != nil {
return err
}
presp, err := handler.masterClient.ShowPartitions(context.TODO(), &milvuspb.ShowPartitionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowPartitions,
MsgID: -1, // todo
Timestamp: 0, // todo
SourceID: Params.NodeID,
},
DbName: "",
CollectionName: schema.Name,
CollectionID: msg.CollectionID,
})
if err = VerifyResponse(presp, err); err != nil {
return err
}
err = handler.meta.AddCollection(&datapb.CollectionInfo{
ID: msg.CollectionID,
Schema: schema,
Partitions: presp.PartitionIDs,
})
if err != nil {
return err
}
return nil
}
func (handler *ddHandler) handleDropCollection(ctx context.Context, msg *msgstream.DropCollectionMsg) error {
segmentsOfCollection := handler.meta.GetSegmentsOfCollection(msg.CollectionID)
for _, id := range segmentsOfCollection {
handler.segmentAllocator.DropSegment(ctx, id)
}
if err := handler.meta.DropCollection(msg.CollectionID); err != nil {
return err
}
return nil
}
func (handler *ddHandler) handleDropPartition(ctx context.Context, msg *msgstream.DropPartitionMsg) error {
segmentsOfPartition := handler.meta.GetSegmentsOfPartition(msg.CollectionID, msg.PartitionID)
for _, id := range segmentsOfPartition {
handler.segmentAllocator.DropSegment(ctx, id)
}
if err := handler.meta.DropPartition(msg.CollectionID, msg.PartitionID); err != nil {
return err
}
return nil
}
func (handler *ddHandler) handleCreatePartition(msg *msgstream.CreatePartitionMsg) error {
return handler.meta.AddPartition(msg.CollectionID, msg.PartitionID)
}
......@@ -101,6 +101,7 @@ func (c *mockDataNodeClient) Stop() error {
}
type mockMasterService struct {
cnt int64
}
func newMockMasterService() *mockMasterService {
......@@ -153,7 +154,14 @@ func (m *mockMasterService) HasCollection(ctx context.Context, req *milvuspb.Has
}
func (m *mockMasterService) DescribeCollection(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
panic("not implemented") // TODO: Implement
return &milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
},
Schema: nil,
CollectionID: 0,
}, nil
}
func (m *mockMasterService) ShowCollections(ctx context.Context, req *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) {
......@@ -197,11 +205,29 @@ func (m *mockMasterService) DropIndex(ctx context.Context, req *milvuspb.DropInd
//global timestamp allocator
func (m *mockMasterService) AllocTimestamp(ctx context.Context, req *masterpb.AllocTimestampRequest) (*masterpb.AllocTimestampResponse, error) {
panic("not implemented") // TODO: Implement
val := atomic.AddInt64(&m.cnt, int64(req.Count))
phy := time.Now().UnixNano() / int64(time.Millisecond)
ts := tsoutil.ComposeTS(phy, val)
return &masterpb.AllocTimestampResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Timestamp: ts,
Count: req.Count,
}, nil
}
func (m *mockMasterService) AllocID(ctx context.Context, req *masterpb.AllocIDRequest) (*masterpb.AllocIDResponse, error) {
panic("not implemented") // TODO: Implement
val := atomic.AddInt64(&m.cnt, int64(req.Count))
return &masterpb.AllocIDResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
ID: val,
Count: req.Count,
}, nil
}
//segment
......
......@@ -48,7 +48,6 @@ type Server struct {
meta *meta
segAllocator segmentAllocatorInterface
statsHandler *statsHandler
ddHandler *ddHandler
allocator allocatorInterface
cluster *dataNodeCluster
msgProducer *timesync.MsgProducer
......@@ -82,10 +81,10 @@ func CreateServer(ctx context.Context, factory msgstream.Factory) (*Server, erro
}
func (s *Server) getInsertChannels() []string {
channels := make([]string, Params.InsertChannelNum)
channels := make([]string, 0, Params.InsertChannelNum)
var i int64 = 0
for ; i < Params.InsertChannelNum; i++ {
channels[i] = Params.InsertChannelPrefixName + strconv.FormatInt(i, 10)
channels = append(channels, Params.InsertChannelPrefixName+strconv.FormatInt(i, 10))
}
return channels
}
......@@ -116,7 +115,6 @@ func (s *Server) Start() error {
s.allocator = newAllocator(s.masterClient)
s.statsHandler = newStatsHandler(s.meta)
s.ddHandler = newDDHandler(s.meta, s.segAllocator, s.masterClient)
s.initSegmentInfoChannel()
s.segAllocator = newSegmentAllocator(s.meta, s.allocator, WithSegmentStream(s.segmentInfoStream))
if err = s.loadMetaFromMaster(); err != nil {
......@@ -317,6 +315,9 @@ func (s *Server) startStatsChannel(ctx context.Context) {
default:
}
msgPack := statsStream.Consume()
if msgPack == nil {
continue
}
for _, msg := range msgPack.Msgs {
statistics, ok := msg.(*msgstream.SegmentStatisticsMsg)
if !ok {
......@@ -348,6 +349,9 @@ func (s *Server) startSegmentFlushChannel(ctx context.Context) {
default:
}
msgPack := flushStream.Consume()
if msgPack == nil {
continue
}
for _, msg := range msgPack.Msgs {
if msg.Type() != commonpb.MsgType_SegmentFlushDone {
continue
......@@ -395,30 +399,6 @@ func (s *Server) startProxyServiceTimeTickLoop(ctx context.Context) {
}
}
func (s *Server) startDDChannel(ctx context.Context) {
defer s.serverLoopWg.Done()
ddStream, _ := s.msFactory.NewMsgStream(ctx)
ddStream.AsConsumer([]string{s.ddChannelMu.name}, Params.DataServiceSubscriptionName)
log.Debug("dataservice AsConsumer: " + s.ddChannelMu.name + " : " + Params.DataServiceSubscriptionName)
ddStream.Start()
defer ddStream.Close()
for {
select {
case <-ctx.Done():
log.Debug("dd channel shut down")
return
default:
}
msgPack := ddStream.Consume()
for _, msg := range msgPack.Msgs {
if err := s.ddHandler.HandleDDMsg(ctx, msg); err != nil {
log.Error("handle dd msg error", zap.Error(err))
continue
}
}
}
}
func (s *Server) Stop() error {
s.cluster.ShutDownClients()
s.ttMsgStream.Close()
......@@ -430,6 +410,11 @@ func (s *Server) Stop() error {
return nil
}
// CleanMeta only for test
func (s *Server) CleanMeta() error {
return s.client.RemoveWithPrefix("")
}
func (s *Server) stopServerLoop() {
s.serverLoopCancel()
s.serverLoopWg.Wait()
......@@ -566,31 +551,33 @@ func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*commonpb
}
func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
resp := &datapb.AssignSegmentIDResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}
if !s.checkStateIsHealthy() {
resp.Status.Reason = "server is initializing"
return resp, nil
return &datapb.AssignSegmentIDResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, nil
}
assigns := make([]*datapb.SegmentIDAssignment, 0, len(req.SegmentIDRequests))
var appendFailedAssignment = func(err string) {
assigns = append(assigns, &datapb.SegmentIDAssignment{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: err,
},
})
}
for _, r := range req.SegmentIDRequests {
if !s.meta.HasCollection(r.CollectionID) {
if err := s.loadCollectionFromMaster(ctx, r.CollectionID); err != nil {
appendFailedAssignment(fmt.Sprintf("can not load collection %d", r.CollectionID))
log.Error("load collection from master error", zap.Int64("collectionID", r.CollectionID), zap.Error(err))
continue
}
}
result := &datapb.SegmentIDAssignment{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}
//if err := s.validateAllocRequest(r.CollectionID, r.PartitionID, r.ChannelName); err != nil {
//result.Status.Reason = err.Error()
//assigns = append(assigns, result)
......@@ -599,24 +586,31 @@ func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentI
segmentID, retCount, expireTs, err := s.segAllocator.AllocSegment(ctx, r.CollectionID, r.PartitionID, r.ChannelName, int(r.Count))
if err != nil {
result.Status.Reason = fmt.Sprintf("allocation of collection %d, partition %d, channel %s, count %d error: %s",
r.CollectionID, r.PartitionID, r.ChannelName, r.Count, err.Error())
assigns = append(assigns, result)
appendFailedAssignment(fmt.Sprintf("allocation of collection %d, partition %d, channel %s, count %d error: %s",
r.CollectionID, r.PartitionID, r.ChannelName, r.Count, err.Error()))
continue
}
result.Status.ErrorCode = commonpb.ErrorCode_Success
result.CollectionID = r.CollectionID
result.SegID = segmentID
result.PartitionID = r.PartitionID
result.Count = uint32(retCount)
result.ExpireTime = expireTs
result.ChannelName = r.ChannelName
result := &datapb.SegmentIDAssignment{
SegID: segmentID,
ChannelName: r.ChannelName,
Count: uint32(retCount),
CollectionID: r.CollectionID,
PartitionID: r.PartitionID,
ExpireTime: expireTs,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}
assigns = append(assigns, result)
}
resp.Status.ErrorCode = commonpb.ErrorCode_Success
resp.SegIDAssignments = assigns
return resp, nil
return &datapb.AssignSegmentIDResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
SegIDAssignments: assigns,
}, nil
}
func (s *Server) validateAllocRequest(collID UniqueID, partID UniqueID, channelName string) error {
......
......@@ -2,16 +2,335 @@ package dataservice
import (
"context"
"math"
"testing"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/types"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
func TestRegisterNode(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
t.Run("register node", func(t *testing.T) {
resp, err := svr.RegisterNode(context.TODO(), &datapb.RegisterNodeRequest{
Base: &commonpb.MsgBase{
MsgType: 0,
MsgID: 0,
Timestamp: 0,
SourceID: 1000,
},
Address: &commonpb.Address{
Ip: "localhost",
Port: 1000,
},
})
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, Params.DataNodeNum, svr.cluster.GetNumOfNodes())
assert.EqualValues(t, []int64{1000}, svr.cluster.GetNodeIDs())
})
}
func TestGetSegmentInfoChannel(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
t.Run("get segment info channel", func(t *testing.T) {
resp, err := svr.GetSegmentInfoChannel(context.TODO())
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, Params.SegmentInfoChannelName, resp.Value)
})
}
func TestGetInsertChannels(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
t.Run("get insert channels", func(t *testing.T) {
resp, err := svr.GetInsertChannels(context.TODO(), &datapb.GetInsertChannelsRequest{
Base: &commonpb.MsgBase{
MsgType: 0,
MsgID: 0,
Timestamp: 0,
SourceID: 1000,
},
DbID: 0,
CollectionID: 0,
})
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, svr.getInsertChannels(), resp.Values)
})
}
func TestAssignSegmentID(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
schema := newTestSchema()
svr.meta.AddCollection(&datapb.CollectionInfo{
ID: 0,
Schema: schema,
Partitions: []int64{},
})
recordSize, err := typeutil.EstimateSizePerRecord(schema)
assert.Nil(t, err)
maxCount := int(Params.SegmentSize * 1024 * 1024 / float64(recordSize))
cases := []struct {
Description string
CollectionID UniqueID
PartitionID UniqueID
ChannelName string
Count uint32
IsSuccess bool
}{
{"assign segment normally", 0, 0, "channel0", 1000, true},
{"assign segment with unexisted collection", 1, 0, "channel0", 1000, false},
{"assign with max count", 0, 0, "channel0", uint32(maxCount), true},
{"assign with max uint32 count", 0, 0, "channel1", math.MaxUint32, false},
}
for _, test := range cases {
t.Run(test.Description, func(t *testing.T) {
req := &datapb.SegmentIDRequest{
Count: test.Count,
ChannelName: test.ChannelName,
CollectionID: test.CollectionID,
PartitionID: test.PartitionID,
}
resp, err := svr.AssignSegmentID(context.TODO(), &datapb.AssignSegmentIDRequest{
NodeID: 0,
PeerRole: "",
SegmentIDRequests: []*datapb.SegmentIDRequest{req},
})
assert.Nil(t, err)
assert.EqualValues(t, 1, len(resp.SegIDAssignments))
assign := resp.SegIDAssignments[0]
if test.IsSuccess {
assert.EqualValues(t, commonpb.ErrorCode_Success, assign.Status.ErrorCode)
assert.EqualValues(t, test.CollectionID, assign.CollectionID)
assert.EqualValues(t, test.PartitionID, assign.PartitionID)
assert.EqualValues(t, test.ChannelName, assign.ChannelName)
assert.EqualValues(t, test.Count, assign.Count)
} else {
assert.NotEqualValues(t, commonpb.ErrorCode_Success, assign.Status.ErrorCode)
}
})
}
}
func TestShowSegments(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
segments := []struct {
id UniqueID
collectionID UniqueID
partitionID UniqueID
}{
{0, 0, 0},
{1, 0, 0},
{2, 0, 1},
{3, 1, 1},
}
for _, segment := range segments {
err := svr.meta.AddSegment(&datapb.SegmentInfo{
ID: segment.id,
CollectionID: segment.collectionID,
PartitionID: segment.partitionID,
})
assert.Nil(t, err)
}
cases := []struct {
description string
collectionID UniqueID
partitionID UniqueID
expected []UniqueID
}{
{"show segments normally", 0, 0, []UniqueID{0, 1}},
{"show non-existed segments", 1, 2, []UniqueID{}},
}
for _, test := range cases {
t.Run(test.description, func(t *testing.T) {
resp, err := svr.ShowSegments(context.TODO(), &datapb.ShowSegmentsRequest{
Base: &commonpb.MsgBase{
MsgType: 0,
MsgID: 0,
Timestamp: 0,
SourceID: 0,
},
CollectionID: test.collectionID,
PartitionID: test.partitionID,
DbID: 0,
})
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, test.expected, resp.SegmentIDs)
})
}
}
func TestFlush(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
schema := newTestSchema()
err := svr.meta.AddCollection(&datapb.CollectionInfo{
ID: 0,
Schema: schema,
Partitions: []int64{},
})
assert.Nil(t, err)
segments := []struct {
id UniqueID
collectionID UniqueID
}{
{1, 0},
{2, 0},
}
for _, segment := range segments {
err = svr.segAllocator.OpenSegment(context.TODO(), &datapb.SegmentInfo{
ID: segment.id,
CollectionID: segment.collectionID,
PartitionID: 0,
State: commonpb.SegmentState_Growing,
})
assert.Nil(t, err)
}
resp, err := svr.Flush(context.TODO(), &datapb.FlushRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Flush,
MsgID: 0,
Timestamp: 0,
SourceID: 0,
},
DbID: 0,
CollectionID: 0,
})
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.ErrorCode)
ids, err := svr.segAllocator.GetSealedSegments(context.TODO())
assert.Nil(t, err)
assert.EqualValues(t, []UniqueID{1, 2}, ids)
}
func TestGetComponentStates(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
cli := newMockDataNodeClient(1)
err := cli.Init()
assert.Nil(t, err)
err = cli.Start()
assert.Nil(t, err)
err = svr.cluster.Register(&dataNode{
id: 1,
address: struct {
ip string
port int64
}{
ip: "",
port: 0,
},
client: cli,
channelNum: 0,
})
assert.Nil(t, err)
resp, err := svr.GetComponentStates(context.TODO())
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, internalpb.StateCode_Healthy, resp.State.StateCode)
assert.EqualValues(t, 1, len(resp.SubcomponentStates))
assert.EqualValues(t, internalpb.StateCode_Healthy, resp.SubcomponentStates[0].StateCode)
}
func TestGetTimeTickChannel(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
resp, err := svr.GetTimeTickChannel(context.TODO())
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, Params.TimeTickChannelName, resp.Value)
}
func TestGetStatisticsChannel(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
resp, err := svr.GetStatisticsChannel(context.TODO())
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, Params.StatisticsChannelName, resp.Value)
}
func TestGetSegmentStates(t *testing.T) {
svr := newTestServer(t)
defer closeTestServer(t, svr)
err := svr.meta.AddSegment(&datapb.SegmentInfo{
ID: 1000,
CollectionID: 100,
PartitionID: 0,
InsertChannel: "",
OpenTime: 0,
SealedTime: 0,
FlushedTime: 0,
NumRows: 0,
MemSize: 0,
State: commonpb.SegmentState_Growing,
StartPosition: &internalpb.MsgPosition{
ChannelName: "",
MsgID: []byte{},
MsgGroup: "",
Timestamp: 0,
},
EndPosition: &internalpb.MsgPosition{
ChannelName: "",
MsgID: []byte{},
MsgGroup: "",
Timestamp: 0,
},
})
assert.Nil(t, err)
cases := []struct {
description string
id UniqueID
expected bool
expectedState commonpb.SegmentState
}{
{"get existed segment", 1000, true, commonpb.SegmentState_Growing},
{"get non-existed segment", 10, false, commonpb.SegmentState_Growing},
}
for _, test := range cases {
t.Run(test.description, func(t *testing.T) {
resp, err := svr.GetSegmentStates(context.TODO(), &datapb.GetSegmentStatesRequest{
Base: &commonpb.MsgBase{
MsgType: 0,
MsgID: 0,
Timestamp: 0,
SourceID: 0,
},
SegmentIDs: []int64{test.id},
})
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, 1, len(resp.States))
if test.expected {
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.States[0].Status.ErrorCode)
assert.EqualValues(t, test.expectedState, resp.States[0].State)
}
})
}
}
func newTestServer(t *testing.T) *Server {
Params.Init()
Params.DataNodeNum = 1
var err error
......@@ -40,23 +359,12 @@ func TestRegisterNode(t *testing.T) {
assert.Nil(t, err)
err = svr.Start()
assert.Nil(t, err)
defer svr.Stop()
t.Run("register node", func(t *testing.T) {
resp, err := svr.RegisterNode(context.TODO(), &datapb.RegisterNodeRequest{
Base: &commonpb.MsgBase{
MsgType: 0,
MsgID: 0,
Timestamp: 0,
SourceID: 1000,
},
Address: &commonpb.Address{
Ip: "localhost",
Port: 1000,
},
})
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.EqualValues(t, Params.DataNodeNum, svr.cluster.GetNumOfNodes())
assert.EqualValues(t, []int64{1000}, svr.cluster.GetNodeIDs())
})
return svr
}
func closeTestServer(t *testing.T, svr *Server) {
err := svr.Stop()
assert.Nil(t, err)
err = svr.CleanMeta()
assert.Nil(t, err)
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册