2022-12-12 10:57:22 +08:00
|
|
|
// Licensed to the LF AI & Data foundation under one
|
|
|
|
// or more contributor license agreements. See the NOTICE file
|
|
|
|
// distributed with this work for additional information
|
|
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
|
|
// to you under the Apache License, Version 2.0 (the
|
|
|
|
// "License"); you may not use this file except in compliance
|
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
//
|
|
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
// See the License for the specific language governing permissions and
|
|
|
|
// limitations under the License.
|
|
|
|
|
|
|
|
// Package datanode implements data persistence logic.
|
|
|
|
//
|
|
|
|
// Data node persists insert logs into persistent storage like minIO/S3.
|
|
|
|
package datanode
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"fmt"
|
|
|
|
"path"
|
|
|
|
"strconv"
|
2023-07-24 14:23:00 +08:00
|
|
|
"time"
|
2022-12-12 10:57:22 +08:00
|
|
|
|
2023-02-26 11:31:49 +08:00
|
|
|
"github.com/cockroachdb/errors"
|
2023-03-04 23:21:50 +08:00
|
|
|
"go.uber.org/zap"
|
2023-02-26 11:31:49 +08:00
|
|
|
|
2023-06-09 01:28:37 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
2024-01-31 20:45:04 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/datanode/importv2"
|
2023-11-30 14:30:28 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/datanode/io"
|
2023-11-15 15:24:18 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
2024-01-18 22:06:31 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
2022-12-12 10:57:22 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/storage"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/importutil"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/common"
|
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
|
|
|
"github.com/milvus-io/milvus/pkg/metrics"
|
2024-01-30 10:09:03 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/tracer"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/metautil"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/retry"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
2023-09-21 09:45:27 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
2022-12-12 10:57:22 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
// WatchDmChannels is not in use
|
|
|
|
func (node *DataNode) WatchDmChannels(ctx context.Context, in *datapb.WatchDmChannelsRequest) (*commonpb.Status, error) {
|
|
|
|
log.Warn("DataNode WatchDmChannels is not in use")
|
|
|
|
|
2023-03-28 19:04:00 +08:00
|
|
|
// TODO ERROR OF GRPC NOT IN USE
|
2023-10-11 21:01:35 +08:00
|
|
|
return merr.Success(), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// GetComponentStates will return current state of DataNode
|
2023-09-26 09:57:25 +08:00
|
|
|
func (node *DataNode) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
|
2022-12-12 10:57:22 +08:00
|
|
|
nodeID := common.NotRegisteredID
|
2023-11-22 09:12:22 +08:00
|
|
|
state := node.stateCode.Load().(commonpb.StateCode)
|
|
|
|
log.Debug("DataNode current state", zap.String("State", state.String()))
|
2023-01-12 19:49:40 +08:00
|
|
|
if node.GetSession() != nil && node.session.Registered() {
|
|
|
|
nodeID = node.GetSession().ServerID
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
states := &milvuspb.ComponentStates{
|
|
|
|
State: &milvuspb.ComponentInfo{
|
|
|
|
// NodeID: Params.NodeID, // will race with DataNode.Register()
|
|
|
|
NodeID: nodeID,
|
|
|
|
Role: node.Role,
|
|
|
|
StateCode: node.stateCode.Load().(commonpb.StateCode),
|
|
|
|
},
|
|
|
|
SubcomponentStates: make([]*milvuspb.ComponentInfo, 0),
|
2023-10-11 21:01:35 +08:00
|
|
|
Status: merr.Success(),
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
return states, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// FlushSegments packs flush messages into flowGraph through flushChan.
|
|
|
|
//
|
|
|
|
// DataCoord calls FlushSegments if the segment is seal&flush only.
|
|
|
|
// If DataNode receives a valid segment to flush, new flush message for the segment should be ignored.
|
|
|
|
// So if receiving calls to flush segment A, DataNode should guarantee the segment to be flushed.
|
|
|
|
func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
|
|
|
|
metrics.DataNodeFlushReqCounter.WithLabelValues(
|
|
|
|
fmt.Sprint(paramtable.GetNodeID()),
|
2023-06-26 17:52:44 +08:00
|
|
|
metrics.TotalLabel).Inc()
|
2022-12-12 10:57:22 +08:00
|
|
|
|
2023-11-15 15:24:18 +08:00
|
|
|
log := log.Ctx(ctx)
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
2023-03-28 19:04:00 +08:00
|
|
|
log.Warn("DataNode.FlushSegments failed", zap.Int64("nodeId", paramtable.GetNodeID()), zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-01-12 19:49:40 +08:00
|
|
|
serverID := node.GetSession().ServerID
|
|
|
|
if req.GetBase().GetTargetID() != serverID {
|
2022-12-12 10:57:22 +08:00
|
|
|
log.Warn("flush segment target id not matched",
|
|
|
|
zap.Int64("targetID", req.GetBase().GetTargetID()),
|
2023-01-12 19:49:40 +08:00
|
|
|
zap.Int64("serverID", serverID),
|
2022-12-12 10:57:22 +08:00
|
|
|
)
|
2023-03-28 19:04:00 +08:00
|
|
|
|
|
|
|
return merr.Status(merr.WrapErrNodeNotMatch(req.GetBase().GetTargetID(), serverID)), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-11-15 15:24:18 +08:00
|
|
|
segmentIDs := req.GetSegmentIDs()
|
|
|
|
log = log.With(
|
2023-11-23 14:40:23 +08:00
|
|
|
zap.Int64("collectionID", req.GetCollectionID()),
|
2023-11-15 15:24:18 +08:00
|
|
|
zap.String("channelName", req.GetChannelName()),
|
|
|
|
zap.Int64s("segmentIDs", segmentIDs),
|
|
|
|
)
|
|
|
|
|
2023-11-23 14:40:23 +08:00
|
|
|
log.Info("receiving FlushSegments request")
|
2022-12-12 10:57:22 +08:00
|
|
|
|
2024-01-31 19:03:04 +08:00
|
|
|
err := node.writeBufferManager.SealSegments(ctx, req.GetChannelName(), segmentIDs)
|
2023-11-15 15:24:18 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to flush segments", zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Log success flushed segments.
|
2023-11-23 14:40:23 +08:00
|
|
|
log.Info("sending segments to WriteBuffer Manager")
|
2022-12-12 10:57:22 +08:00
|
|
|
|
|
|
|
metrics.DataNodeFlushReqCounter.WithLabelValues(
|
|
|
|
fmt.Sprint(paramtable.GetNodeID()),
|
2023-06-26 17:52:44 +08:00
|
|
|
metrics.SuccessLabel).Inc()
|
2023-10-11 21:01:35 +08:00
|
|
|
return merr.Success(), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-10-16 10:26:09 +08:00
|
|
|
// ResendSegmentStats . ResendSegmentStats resend un-flushed segment stats back upstream to DataCoord by resending DataNode time tick message.
|
2022-12-12 10:57:22 +08:00
|
|
|
// It returns a list of segments to be sent.
|
2023-10-16 10:26:09 +08:00
|
|
|
// Deprecated in 2.3.2, reversed it just for compatibility during rolling back
|
2022-12-12 10:57:22 +08:00
|
|
|
func (node *DataNode) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) {
|
|
|
|
return &datapb.ResendSegmentStatsResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
Status: merr.Success(),
|
2023-10-16 10:26:09 +08:00
|
|
|
SegResent: make([]int64, 0),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// GetTimeTickChannel currently do nothing
|
2023-09-26 09:57:25 +08:00
|
|
|
func (node *DataNode) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
|
2022-12-12 10:57:22 +08:00
|
|
|
return &milvuspb.StringResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
Status: merr.Success(),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// GetStatisticsChannel currently do nothing
|
2023-09-26 09:57:25 +08:00
|
|
|
func (node *DataNode) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
|
2022-12-12 10:57:22 +08:00
|
|
|
return &milvuspb.StringResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
Status: merr.Success(),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// ShowConfigurations returns the configurations of DataNode matching req.Pattern
|
|
|
|
func (node *DataNode) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
|
|
|
|
log.Debug("DataNode.ShowConfigurations", zap.String("pattern", req.Pattern))
|
2023-10-11 21:01:35 +08:00
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
2023-03-28 19:04:00 +08:00
|
|
|
log.Warn("DataNode.ShowConfigurations failed", zap.Int64("nodeId", paramtable.GetNodeID()), zap.Error(err))
|
2022-12-12 10:57:22 +08:00
|
|
|
|
|
|
|
return &internalpb.ShowConfigurationsResponse{
|
2023-03-28 19:04:00 +08:00
|
|
|
Status: merr.Status(err),
|
2022-12-12 10:57:22 +08:00
|
|
|
Configuations: nil,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
configList := make([]*commonpb.KeyValuePair, 0)
|
2023-01-13 15:31:41 +08:00
|
|
|
for key, value := range Params.GetComponentConfigurations("datanode", req.Pattern) {
|
2022-12-12 10:57:22 +08:00
|
|
|
configList = append(configList,
|
|
|
|
&commonpb.KeyValuePair{
|
|
|
|
Key: key,
|
|
|
|
Value: value,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
return &internalpb.ShowConfigurationsResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
Status: merr.Success(),
|
2022-12-12 10:57:22 +08:00
|
|
|
Configuations: configList,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// GetMetrics return datanode metrics
|
|
|
|
func (node *DataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
2023-10-11 21:01:35 +08:00
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
2023-03-28 19:04:00 +08:00
|
|
|
log.Warn("DataNode.GetMetrics failed", zap.Int64("nodeId", paramtable.GetNodeID()), zap.Error(err))
|
2022-12-12 10:57:22 +08:00
|
|
|
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
2023-03-28 19:04:00 +08:00
|
|
|
Status: merr.Status(err),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
metricType, err := metricsinfo.ParseMetricType(req.Request)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("DataNode.GetMetrics failed to parse metric type",
|
2023-01-04 17:39:35 +08:00
|
|
|
zap.Int64("nodeID", paramtable.GetNodeID()),
|
2022-12-12 10:57:22 +08:00
|
|
|
zap.String("req", req.Request),
|
|
|
|
zap.Error(err))
|
|
|
|
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
2023-03-28 19:04:00 +08:00
|
|
|
Status: merr.Status(err),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if metricType == metricsinfo.SystemInfoMetrics {
|
|
|
|
systemInfoMetrics, err := node.getSystemInfoMetrics(ctx, req)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("DataNode GetMetrics failed", zap.Int64("nodeID", paramtable.GetNodeID()), zap.Error(err))
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
2023-03-28 19:04:00 +08:00
|
|
|
Status: merr.Status(err),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
return systemInfoMetrics, nil
|
|
|
|
}
|
|
|
|
|
2023-01-04 17:39:35 +08:00
|
|
|
log.RatedWarn(60, "DataNode.GetMetrics failed, request metric type is not implemented yet",
|
|
|
|
zap.Int64("nodeID", paramtable.GetNodeID()),
|
2022-12-12 10:57:22 +08:00
|
|
|
zap.String("req", req.Request),
|
|
|
|
zap.String("metric_type", metricType))
|
|
|
|
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
2023-03-28 19:04:00 +08:00
|
|
|
Status: merr.Status(merr.WrapErrMetricNotFound(metricType)),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Compaction handles compaction request from DataCoord
|
|
|
|
// returns status as long as compaction task enqueued or invalid
|
|
|
|
func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) {
|
2023-12-04 19:06:40 +08:00
|
|
|
log := log.Ctx(ctx).With(zap.Int64("planID", req.GetPlanID()))
|
2023-10-11 21:01:35 +08:00
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
2023-03-28 19:04:00 +08:00
|
|
|
log.Warn("DataNode.Compaction failed", zap.Int64("nodeId", paramtable.GetNodeID()), zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-11-30 18:42:32 +08:00
|
|
|
ds, ok := node.flowgraphManager.GetFlowgraphService(req.GetChannel())
|
2022-12-12 10:57:22 +08:00
|
|
|
if !ok {
|
2023-07-14 15:56:31 +08:00
|
|
|
log.Warn("illegel compaction plan, channel not in this DataNode", zap.String("channelName", req.GetChannel()))
|
2023-03-28 19:04:00 +08:00
|
|
|
return merr.Status(merr.WrapErrChannelNotFound(req.GetChannel(), "illegel compaction plan")), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-11-29 10:50:29 +08:00
|
|
|
if !node.compactionExecutor.isValidChannel(req.GetChannel()) {
|
2023-07-14 15:56:31 +08:00
|
|
|
log.Warn("channel of compaction is marked invalid in compaction executor", zap.String("channelName", req.GetChannel()))
|
2023-03-28 19:04:00 +08:00
|
|
|
return merr.Status(merr.WrapErrChannelNotFound(req.GetChannel(), "channel is dropping")), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-12-04 19:06:40 +08:00
|
|
|
meta := ds.metacache
|
|
|
|
for _, segment := range req.GetSegmentBinlogs() {
|
|
|
|
if segment.GetLevel() == datapb.SegmentLevel_L0 {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
_, ok := meta.GetSegmentByID(segment.GetSegmentID(), metacache.WithSegmentState(commonpb.SegmentState_Flushed))
|
|
|
|
if !ok {
|
|
|
|
log.Warn("compaction plan contains segment which is not flushed",
|
|
|
|
zap.Int64("segmentID", segment.GetSegmentID()),
|
|
|
|
)
|
|
|
|
return merr.Status(merr.WrapErrSegmentNotFound(segment.GetSegmentID(), "segment with flushed state not found")), nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2024-01-30 10:09:03 +08:00
|
|
|
/*
|
|
|
|
spanCtx := trace.SpanContextFromContext(ctx)
|
2024-01-23 10:37:00 +08:00
|
|
|
|
2024-01-30 10:09:03 +08:00
|
|
|
taskCtx := trace.ContextWithSpanContext(node.ctx, spanCtx)*/
|
|
|
|
taskCtx := tracer.Propagate(ctx, node.ctx)
|
2024-01-23 10:37:00 +08:00
|
|
|
|
2023-11-30 14:30:28 +08:00
|
|
|
var task compactor
|
|
|
|
switch req.GetType() {
|
|
|
|
case datapb.CompactionType_Level0DeleteCompaction:
|
|
|
|
binlogIO := io.NewBinlogIO(node.chunkManager, getOrCreateIOPool())
|
|
|
|
task = newLevelZeroCompactionTask(
|
2024-01-23 10:37:00 +08:00
|
|
|
taskCtx,
|
2023-11-30 14:30:28 +08:00
|
|
|
binlogIO,
|
|
|
|
node.allocator,
|
|
|
|
ds.metacache,
|
|
|
|
node.syncMgr,
|
|
|
|
req,
|
|
|
|
)
|
2024-02-01 14:25:04 +08:00
|
|
|
case datapb.CompactionType_MixCompaction:
|
2024-02-20 14:38:51 +08:00
|
|
|
binlogIO := io.NewBinlogIO(node.chunkManager, getOrCreateIOPool())
|
2023-11-30 14:30:28 +08:00
|
|
|
task = newCompactionTask(
|
2024-01-23 10:37:00 +08:00
|
|
|
taskCtx,
|
2024-02-20 14:38:51 +08:00
|
|
|
binlogIO,
|
2023-11-30 14:30:28 +08:00
|
|
|
ds.metacache,
|
|
|
|
node.syncMgr,
|
|
|
|
node.allocator,
|
|
|
|
req,
|
|
|
|
)
|
|
|
|
default:
|
|
|
|
log.Warn("Unknown compaction type", zap.String("type", req.GetType().String()))
|
|
|
|
return merr.Status(merr.WrapErrParameterInvalidMsg("Unknown compaction type: %v", req.GetType().String())), nil
|
|
|
|
}
|
2022-12-12 10:57:22 +08:00
|
|
|
|
|
|
|
node.compactionExecutor.execute(task)
|
2023-10-11 21:01:35 +08:00
|
|
|
return merr.Success(), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// GetCompactionState called by DataCoord
|
|
|
|
// return status of all compaction plans
|
|
|
|
func (node *DataNode) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
|
2023-10-11 21:01:35 +08:00
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
2023-03-28 19:04:00 +08:00
|
|
|
log.Warn("DataNode.GetCompactionState failed", zap.Int64("nodeId", paramtable.GetNodeID()), zap.Error(err))
|
2022-12-12 10:57:22 +08:00
|
|
|
return &datapb.CompactionStateResponse{
|
2023-03-28 19:04:00 +08:00
|
|
|
Status: merr.Status(err),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
2023-11-29 10:50:29 +08:00
|
|
|
results := node.compactionExecutor.getAllCompactionResults()
|
2022-12-12 10:57:22 +08:00
|
|
|
return &datapb.CompactionStateResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
Status: merr.Success(),
|
2022-12-12 10:57:22 +08:00
|
|
|
Results: results,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// SyncSegments called by DataCoord, sync the compacted segments' meta between DC and DN
|
|
|
|
func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
|
|
|
|
log.Ctx(ctx).Info("DataNode receives SyncSegments",
|
|
|
|
zap.Int64("planID", req.GetPlanID()),
|
|
|
|
zap.Int64("target segmentID", req.GetCompactedTo()),
|
|
|
|
zap.Int64s("compacted from", req.GetCompactedFrom()),
|
|
|
|
zap.Int64("numOfRows", req.GetNumOfRows()),
|
2023-11-07 10:06:17 +08:00
|
|
|
zap.String("channelName", req.GetChannelName()),
|
2022-12-12 10:57:22 +08:00
|
|
|
)
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
2023-03-28 19:04:00 +08:00
|
|
|
log.Warn("DataNode.SyncSegments failed", zap.Int64("nodeId", paramtable.GetNodeID()), zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
if len(req.GetCompactedFrom()) <= 0 {
|
2023-03-28 19:04:00 +08:00
|
|
|
return merr.Status(merr.WrapErrParameterInvalid(">0", "0", "compacted from segments shouldn't be empty")), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-11-30 18:42:32 +08:00
|
|
|
ds, ok := node.flowgraphManager.GetFlowgraphService(req.GetChannelName())
|
2023-11-15 15:24:18 +08:00
|
|
|
if !ok {
|
2023-11-29 10:50:29 +08:00
|
|
|
node.compactionExecutor.clearTasksByChannel(req.GetChannelName())
|
2023-11-15 15:24:18 +08:00
|
|
|
err := merr.WrapErrChannelNotFound(req.GetChannelName())
|
|
|
|
log.Warn("failed to sync segments", zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
2024-01-18 22:06:31 +08:00
|
|
|
err := binlog.DecompressBinLog(storage.StatsBinlog, req.GetCollectionId(), req.GetPartitionId(), req.GetCompactedTo(), req.GetStatsLogs())
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to DecompressBinLog", zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
|
|
}
|
|
|
|
pks, err := loadStats(ctx, node.chunkManager, ds.metacache.Schema(), req.GetCompactedTo(), req.GetStatsLogs())
|
2022-12-12 10:57:22 +08:00
|
|
|
if err != nil {
|
2023-11-15 15:24:18 +08:00
|
|
|
log.Warn("failed to load segment statslog", zap.Error(err))
|
2023-03-28 19:04:00 +08:00
|
|
|
return merr.Status(err), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
2023-11-15 15:24:18 +08:00
|
|
|
bfs := metacache.NewBloomFilterSet(pks...)
|
|
|
|
ds.metacache.CompactSegments(req.GetCompactedTo(), req.GetPartitionId(), req.GetNumOfRows(), bfs, req.GetCompactedFrom()...)
|
2023-11-29 10:50:29 +08:00
|
|
|
node.compactionExecutor.injectDone(req.GetPlanID())
|
2023-10-11 21:01:35 +08:00
|
|
|
return merr.Success(), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-09-25 14:05:27 +08:00
|
|
|
func (node *DataNode) NotifyChannelOperation(ctx context.Context, req *datapb.ChannelOperationsRequest) (*commonpb.Status, error) {
|
2023-12-27 16:00:48 +08:00
|
|
|
log.Ctx(ctx).Info("DataNode receives NotifyChannelOperation",
|
|
|
|
zap.Int("operation count", len(req.GetInfos())))
|
|
|
|
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
|
|
log.Warn("DataNode.NotifyChannelOperation failed", zap.Int64("nodeId", paramtable.GetNodeID()), zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, info := range req.GetInfos() {
|
|
|
|
err := node.channelManager.Submit(info)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("Submit error", zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return merr.Status(nil), nil
|
2023-09-25 14:05:27 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (node *DataNode) CheckChannelOperationProgress(ctx context.Context, req *datapb.ChannelWatchInfo) (*datapb.ChannelOperationProgressResponse, error) {
|
2023-12-27 16:00:48 +08:00
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
zap.String("channel", req.GetVchan().GetChannelName()),
|
|
|
|
zap.String("operation", req.GetState().String()),
|
|
|
|
)
|
|
|
|
|
|
|
|
log.Info("DataNode receives CheckChannelOperationProgress")
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
|
|
log.Warn("DataNode.CheckChannelOperationProgress failed", zap.Int64("nodeId", paramtable.GetNodeID()), zap.Error(err))
|
|
|
|
return &datapb.ChannelOperationProgressResponse{
|
|
|
|
Status: merr.Status(err),
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
return node.channelManager.GetProgress(req), nil
|
2023-09-25 14:05:27 +08:00
|
|
|
}
|
|
|
|
|
2022-12-12 10:57:22 +08:00
|
|
|
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
|
|
|
|
func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields := []zap.Field{
|
2022-12-12 10:57:22 +08:00
|
|
|
zap.Int64("task ID", req.GetImportTask().GetTaskId()),
|
2023-07-14 15:56:31 +08:00
|
|
|
zap.Int64("collectionID", req.GetImportTask().GetCollectionId()),
|
|
|
|
zap.Int64("partitionID", req.GetImportTask().GetPartitionId()),
|
2023-07-11 15:18:28 +08:00
|
|
|
zap.String("database name", req.GetImportTask().GetDatabaseName()),
|
2022-12-12 10:57:22 +08:00
|
|
|
zap.Strings("channel names", req.GetImportTask().GetChannelNames()),
|
2023-07-11 15:18:28 +08:00
|
|
|
zap.Int64s("working dataNodes", req.WorkingNodes),
|
|
|
|
zap.Int64("node ID", paramtable.GetNodeID()),
|
|
|
|
}
|
|
|
|
log.Info("DataNode receive import request", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
defer func() {
|
2023-07-11 15:18:28 +08:00
|
|
|
log.Info("DataNode finish import request", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
}()
|
|
|
|
|
|
|
|
importResult := &rootcoordpb.ImportResult{
|
2023-10-11 21:01:35 +08:00
|
|
|
Status: merr.Success(),
|
2022-12-12 10:57:22 +08:00
|
|
|
TaskId: req.GetImportTask().TaskId,
|
|
|
|
DatanodeId: paramtable.GetNodeID(),
|
|
|
|
State: commonpb.ImportState_ImportStarted,
|
|
|
|
Segments: make([]int64, 0),
|
|
|
|
AutoIds: make([]int64, 0),
|
|
|
|
RowCount: 0,
|
|
|
|
}
|
2023-01-11 17:37:44 +08:00
|
|
|
importResult.Infos = append(importResult.Infos, &commonpb.KeyValuePair{Key: importutil.ProgressPercent, Value: "0"})
|
2022-12-12 10:57:22 +08:00
|
|
|
|
|
|
|
// Spawn a new context to ignore cancellation from parental context.
|
2023-07-24 14:23:00 +08:00
|
|
|
newCtx, cancel := context.WithTimeout(context.TODO(), paramtable.Get().DataNodeCfg.BulkInsertTimeoutSeconds.GetAsDuration(time.Second))
|
2022-12-12 10:57:22 +08:00
|
|
|
defer cancel()
|
2023-07-11 15:18:28 +08:00
|
|
|
|
|
|
|
// function to report import state to RootCoord.
|
|
|
|
// retry 10 times, if the rootcoord is down, the report function will cost 20+ seconds
|
|
|
|
reportFunc := reportImportFunc(node)
|
2023-09-18 14:05:28 +08:00
|
|
|
returnFailFunc := func(msg string, err error) (*commonpb.Status, error) {
|
|
|
|
logFields = append(logFields, zap.Error(err))
|
2023-07-11 15:18:28 +08:00
|
|
|
log.Warn(msg, logFields...)
|
|
|
|
importResult.State = commonpb.ImportState_ImportFailed
|
2023-09-18 14:05:28 +08:00
|
|
|
importResult.Infos = append(importResult.Infos, &commonpb.KeyValuePair{Key: importutil.FailedReason, Value: err.Error()})
|
2023-07-11 15:18:28 +08:00
|
|
|
|
|
|
|
reportFunc(importResult)
|
|
|
|
|
2023-09-18 14:05:28 +08:00
|
|
|
return merr.Status(err), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields = append(logFields, zap.Error(err))
|
|
|
|
log.Warn("DataNode import failed, node is not healthy", logFields...)
|
2023-03-28 19:04:00 +08:00
|
|
|
return merr.Status(err), nil
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// get a timestamp for all the rows
|
|
|
|
// Ignore cancellation from parent context.
|
2023-10-13 09:55:34 +08:00
|
|
|
ts, _, err := node.broker.AllocTimestamp(newCtx, 1)
|
|
|
|
if err != nil {
|
2023-07-11 15:18:28 +08:00
|
|
|
return returnFailFunc("DataNode alloc ts failed", err)
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// get collection schema and shard number
|
2023-10-13 09:55:34 +08:00
|
|
|
metaService := newMetaService(node.broker, req.GetImportTask().GetCollectionId())
|
2022-12-12 10:57:22 +08:00
|
|
|
colInfo, err := metaService.getCollectionInfo(newCtx, req.GetImportTask().GetCollectionId(), 0)
|
|
|
|
if err != nil {
|
2023-07-11 15:18:28 +08:00
|
|
|
return returnFailFunc("failed to get collection info for collection ID", err)
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-09-26 10:25:25 +08:00
|
|
|
var partitionIDs []int64
|
|
|
|
if req.GetImportTask().GetPartitionId() == 0 {
|
|
|
|
if !typeutil.HasPartitionKey(colInfo.GetSchema()) {
|
|
|
|
err = errors.New("try auto-distribute data but the collection has no partition key")
|
|
|
|
return returnFailFunc(err.Error(), err)
|
|
|
|
}
|
|
|
|
// TODO: prefer to set partitionIDs in coord instead of get here.
|
|
|
|
// the colInfo doesn't have a correct database name(it is empty). use the database name passed from rootcoord.
|
2023-10-13 09:55:34 +08:00
|
|
|
partitions, err := node.broker.ShowPartitions(ctx, req.GetImportTask().GetDatabaseName(), colInfo.GetCollectionName())
|
2023-09-26 10:25:25 +08:00
|
|
|
if err != nil {
|
|
|
|
return returnFailFunc("failed to get partition id list", err)
|
|
|
|
}
|
|
|
|
_, partitionIDs, err = typeutil.RearrangePartitionsForPartitionKey(partitions)
|
|
|
|
if err != nil {
|
|
|
|
return returnFailFunc("failed to rearrange target partitions", err)
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
partitionIDs = []int64{req.GetImportTask().GetPartitionId()}
|
2023-07-11 15:18:28 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
collectionInfo, err := importutil.NewCollectionInfo(colInfo.GetSchema(), colInfo.GetShardsNum(), partitionIDs)
|
|
|
|
if err != nil {
|
|
|
|
return returnFailFunc("invalid collection info to import", err)
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// parse files and generate segments
|
|
|
|
segmentSize := Params.DataCoordCfg.SegmentMaxSize.GetAsInt64() * 1024 * 1024
|
2023-11-07 15:14:26 +08:00
|
|
|
importWrapper := importutil.NewImportWrapper(newCtx, collectionInfo, segmentSize, Params.DataNodeCfg.BinLogMaxSize.GetAsInt64(),
|
|
|
|
node.allocator.GetIDAlloactor(), node.chunkManager, importResult, reportFunc)
|
2022-12-12 10:57:22 +08:00
|
|
|
importWrapper.SetCallbackFunctions(assignSegmentFunc(node, req),
|
|
|
|
createBinLogsFunc(node, req, colInfo.GetSchema(), ts),
|
|
|
|
saveSegmentFunc(node, req, importResult, ts))
|
|
|
|
// todo: pass tsStart and tsStart after import_wrapper support
|
|
|
|
tsStart, tsEnd, err := importutil.ParseTSFromOptions(req.GetImportTask().GetInfos())
|
|
|
|
isBackup := importutil.IsBackup(req.GetImportTask().GetInfos())
|
|
|
|
if err != nil {
|
2023-07-11 15:18:28 +08:00
|
|
|
return returnFailFunc("failed to parse timestamp from import options", err)
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields = append(logFields, zap.Uint64("start_ts", tsStart), zap.Uint64("end_ts", tsEnd))
|
|
|
|
log.Info("import time range", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
err = importWrapper.Import(req.GetImportTask().GetFiles(),
|
|
|
|
importutil.ImportOptions{OnlyValidate: false, TsStartPoint: tsStart, TsEndPoint: tsEnd, IsBackup: isBackup})
|
|
|
|
if err != nil {
|
2023-07-11 15:18:28 +08:00
|
|
|
return returnFailFunc("failed to import files", err)
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
resp := merr.Success()
|
2023-07-11 15:18:28 +08:00
|
|
|
return resp, nil
|
|
|
|
}
|
|
|
|
|
2023-09-12 21:07:19 +08:00
|
|
|
func (node *DataNode) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
|
|
|
|
log := log.Ctx(ctx).With(zap.Int64("nodeId", paramtable.GetNodeID()),
|
|
|
|
zap.Time("flushTs", tsoutil.PhysicalTime(req.GetFlushTs())),
|
2023-09-16 00:41:20 +08:00
|
|
|
zap.Strings("channels", req.GetChannels()))
|
2023-09-12 21:07:19 +08:00
|
|
|
|
|
|
|
log.Info("DataNode receives FlushChannels request")
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
2023-09-12 21:07:19 +08:00
|
|
|
log.Warn("DataNode.FlushChannels failed", zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, channel := range req.GetChannels() {
|
2023-11-15 15:24:18 +08:00
|
|
|
err := node.writeBufferManager.FlushChannel(ctx, channel, req.GetFlushTs())
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to flush channel", zap.String("channel", channel), zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
2023-09-12 21:07:19 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
return merr.Success(), nil
|
2023-09-12 21:07:19 +08:00
|
|
|
}
|
|
|
|
|
2022-12-12 10:57:22 +08:00
|
|
|
// AddImportSegment adds the import segment to the current DataNode.
|
|
|
|
func (node *DataNode) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields := []zap.Field{
|
2023-07-14 15:56:31 +08:00
|
|
|
zap.Int64("segmentID", req.GetSegmentId()),
|
|
|
|
zap.Int64("collectionID", req.GetCollectionId()),
|
|
|
|
zap.Int64("partitionID", req.GetPartitionId()),
|
|
|
|
zap.String("channelName", req.GetChannelName()),
|
2023-07-11 15:18:28 +08:00
|
|
|
zap.Int64("# of rows", req.GetRowNum()),
|
|
|
|
}
|
|
|
|
log.Info("adding segment to DataNode flow graph", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
// Fetch the flow graph on the given v-channel.
|
|
|
|
var ds *dataSyncService
|
|
|
|
// Retry in case the channel hasn't been watched yet.
|
|
|
|
err := retry.Do(ctx, func() error {
|
|
|
|
var ok bool
|
2023-11-30 18:42:32 +08:00
|
|
|
ds, ok = node.flowgraphManager.GetFlowgraphService(req.GetChannelName())
|
2022-12-12 10:57:22 +08:00
|
|
|
if !ok {
|
2023-07-11 15:18:28 +08:00
|
|
|
return errors.New("channel not found")
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}, retry.Attempts(getFlowGraphServiceAttempts))
|
|
|
|
if err != nil {
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields = append(logFields, zap.Int64("node ID", paramtable.GetNodeID()))
|
|
|
|
log.Error("channel not found in current DataNode", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
return &datapb.AddImportSegmentResponse{
|
2023-07-11 15:18:28 +08:00
|
|
|
Status: &commonpb.Status{
|
|
|
|
// TODO: Add specific error code.
|
|
|
|
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
|
|
|
Reason: "channel not found in current DataNode",
|
|
|
|
},
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
// Add the new segment to the channel.
|
2023-11-15 15:24:18 +08:00
|
|
|
if len(ds.metacache.GetSegmentIDsBy(metacache.WithSegmentIDs(req.GetSegmentId()), metacache.WithSegmentState(commonpb.SegmentState_Flushed))) == 0 {
|
2023-07-11 15:18:28 +08:00
|
|
|
log.Info("adding a new segment to channel", logFields...)
|
2024-01-18 22:06:31 +08:00
|
|
|
// no error will be throw
|
|
|
|
err := binlog.DecompressBinLog(storage.StatsBinlog, req.GetCollectionId(), req.GetPartitionId(), req.GetSegmentId(), req.GetStatsLog())
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to DecompressBinLog", zap.Error(err))
|
|
|
|
return &datapb.AddImportSegmentResponse{
|
|
|
|
Status: merr.Status(err),
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
pks, err := loadStats(ctx, node.chunkManager, ds.metacache.Schema(), req.GetSegmentId(), req.GetStatsLog())
|
2023-11-15 15:24:18 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to get segment pk stats", zap.Error(err))
|
2022-12-12 10:57:22 +08:00
|
|
|
return &datapb.AddImportSegmentResponse{
|
2023-11-15 15:24:18 +08:00
|
|
|
Status: merr.Status(err),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
2023-11-15 15:24:18 +08:00
|
|
|
|
|
|
|
// Add segment as a flushed segment, but set `importing` to true to add extra information of the segment.
|
|
|
|
// By 'extra information' we mean segment info while adding a `SegmentType_Flushed` typed segment.
|
|
|
|
// ds.metacache.
|
|
|
|
ds.metacache.AddSegment(&datapb.SegmentInfo{
|
|
|
|
ID: req.GetSegmentId(),
|
|
|
|
State: commonpb.SegmentState_Flushed,
|
|
|
|
CollectionID: req.GetCollectionId(),
|
|
|
|
PartitionID: req.GetPartitionId(),
|
|
|
|
InsertChannel: req.GetChannelName(),
|
|
|
|
NumOfRows: req.GetRowNum(),
|
|
|
|
Statslogs: req.GetStatsLog(),
|
|
|
|
StartPosition: &msgpb.MsgPosition{
|
|
|
|
ChannelName: req.GetChannelName(),
|
|
|
|
Timestamp: req.GetBase().GetTimestamp(),
|
|
|
|
},
|
|
|
|
DmlPosition: &msgpb.MsgPosition{
|
|
|
|
ChannelName: req.GetChannelName(),
|
|
|
|
Timestamp: req.GetBase().GetTimestamp(),
|
|
|
|
},
|
|
|
|
}, func(info *datapb.SegmentInfo) *metacache.BloomFilterSet {
|
|
|
|
bfs := metacache.NewBloomFilterSet(pks...)
|
|
|
|
return bfs
|
2024-02-03 13:01:12 +08:00
|
|
|
})
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
2023-11-15 15:24:18 +08:00
|
|
|
|
2022-12-12 10:57:22 +08:00
|
|
|
return &datapb.AddImportSegmentResponse{
|
2024-01-22 14:36:55 +08:00
|
|
|
Status: merr.Success(),
|
2022-12-12 10:57:22 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func assignSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest) importutil.AssignSegmentFunc {
|
2023-07-11 15:18:28 +08:00
|
|
|
return func(shardID int, partID int64) (int64, string, error) {
|
2022-12-12 10:57:22 +08:00
|
|
|
chNames := req.GetImportTask().GetChannelNames()
|
|
|
|
importTaskID := req.GetImportTask().GetTaskId()
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields := []zap.Field{
|
|
|
|
zap.Int64("task ID", importTaskID),
|
|
|
|
zap.Int("shard ID", shardID),
|
2023-07-14 15:56:31 +08:00
|
|
|
zap.Int64("partitionID", partID),
|
2023-07-11 15:18:28 +08:00
|
|
|
zap.Int("# of channels", len(chNames)),
|
|
|
|
zap.Strings("channel names", chNames),
|
|
|
|
}
|
2022-12-12 10:57:22 +08:00
|
|
|
if shardID >= len(chNames) {
|
2023-07-11 15:18:28 +08:00
|
|
|
log.Error("import task returns invalid shard ID", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
return 0, "", fmt.Errorf("syncSegmentID Failed: invalid shard ID %d", shardID)
|
|
|
|
}
|
|
|
|
|
2023-07-11 15:18:28 +08:00
|
|
|
tr := timerecord.NewTimeRecorder("assign segment function")
|
2022-12-12 10:57:22 +08:00
|
|
|
defer tr.Elapse("finished")
|
|
|
|
|
|
|
|
colID := req.GetImportTask().GetCollectionId()
|
|
|
|
segmentIDReq := composeAssignSegmentIDRequest(1, shardID, chNames, colID, partID)
|
|
|
|
targetChName := segmentIDReq.GetSegmentIDRequests()[0].GetChannelName()
|
2023-07-25 11:29:00 +08:00
|
|
|
logFields = append(logFields, zap.Int64("collection ID", colID))
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields = append(logFields, zap.String("target channel name", targetChName))
|
|
|
|
log.Info("assign segment for the import task", logFields...)
|
2023-10-13 09:55:34 +08:00
|
|
|
ids, err := node.broker.AssignSegmentID(context.Background(), segmentIDReq.GetSegmentIDRequests()...)
|
2022-12-12 10:57:22 +08:00
|
|
|
if err != nil {
|
2023-10-13 09:55:34 +08:00
|
|
|
return 0, "", errors.Wrap(err, "failed to AssignSegmentID")
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
2023-10-13 09:55:34 +08:00
|
|
|
|
|
|
|
if len(ids) == 0 {
|
|
|
|
return 0, "", merr.WrapErrSegmentNotFound(0, "failed to assign segment id")
|
2023-06-13 10:22:38 +08:00
|
|
|
}
|
2023-07-11 15:18:28 +08:00
|
|
|
|
2023-10-13 09:55:34 +08:00
|
|
|
segmentID := ids[0]
|
2023-07-14 15:56:31 +08:00
|
|
|
logFields = append(logFields, zap.Int64("segmentID", segmentID))
|
2023-07-11 15:18:28 +08:00
|
|
|
log.Info("new segment assigned", logFields...)
|
2023-01-11 11:55:39 +08:00
|
|
|
|
|
|
|
// call report to notify the rootcoord update the segment id list for this task
|
|
|
|
// ignore the returned error, since even report failed the segments still can be cleaned
|
2023-07-11 15:18:28 +08:00
|
|
|
// retry 10 times, if the rootcoord is down, the report function will cost 20+ seconds
|
|
|
|
importResult := &rootcoordpb.ImportResult{
|
2023-10-11 21:01:35 +08:00
|
|
|
Status: merr.Success(),
|
2023-07-11 15:18:28 +08:00
|
|
|
TaskId: req.GetImportTask().TaskId,
|
|
|
|
DatanodeId: paramtable.GetNodeID(),
|
|
|
|
State: commonpb.ImportState_ImportStarted,
|
|
|
|
Segments: []int64{segmentID},
|
|
|
|
AutoIds: make([]int64, 0),
|
|
|
|
RowCount: 0,
|
|
|
|
}
|
|
|
|
reportFunc := reportImportFunc(node)
|
|
|
|
reportFunc(importResult)
|
2023-01-11 11:55:39 +08:00
|
|
|
|
2022-12-12 10:57:22 +08:00
|
|
|
return segmentID, targetChName, nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func createBinLogsFunc(node *DataNode, req *datapb.ImportTaskRequest, schema *schemapb.CollectionSchema, ts Timestamp) importutil.CreateBinlogsFunc {
|
2023-07-11 15:18:28 +08:00
|
|
|
return func(fields importutil.BlockData, segmentID int64, partID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
|
2022-12-12 10:57:22 +08:00
|
|
|
var rowNum int
|
|
|
|
for _, field := range fields {
|
|
|
|
rowNum = field.RowNum()
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
chNames := req.GetImportTask().GetChannelNames()
|
|
|
|
importTaskID := req.GetImportTask().GetTaskId()
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields := []zap.Field{
|
|
|
|
zap.Int64("task ID", importTaskID),
|
2023-07-14 15:56:31 +08:00
|
|
|
zap.Int64("partitionID", partID),
|
|
|
|
zap.Int64("segmentID", segmentID),
|
2023-07-11 15:18:28 +08:00
|
|
|
zap.Int("# of channels", len(chNames)),
|
|
|
|
zap.Strings("channel names", chNames),
|
|
|
|
}
|
|
|
|
|
2022-12-12 10:57:22 +08:00
|
|
|
if rowNum <= 0 {
|
2023-07-11 15:18:28 +08:00
|
|
|
log.Info("fields data is empty, no need to generate binlog", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
return nil, nil, nil
|
|
|
|
}
|
2023-07-25 11:29:00 +08:00
|
|
|
logFields = append(logFields, zap.Int("row count", rowNum))
|
2022-12-12 10:57:22 +08:00
|
|
|
|
|
|
|
colID := req.GetImportTask().GetCollectionId()
|
|
|
|
fieldInsert, fieldStats, err := createBinLogs(rowNum, schema, ts, fields, node, segmentID, colID, partID)
|
|
|
|
if err != nil {
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields = append(logFields, zap.Any("err", err))
|
|
|
|
log.Error("failed to create binlogs", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields = append(logFields, zap.Int("insert log count", len(fieldInsert)), zap.Int("stats log count", len(fieldStats)))
|
|
|
|
log.Info("new binlog created", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
|
|
|
|
return fieldInsert, fieldStats, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func saveSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest, res *rootcoordpb.ImportResult, ts Timestamp) importutil.SaveSegmentFunc {
|
|
|
|
importTaskID := req.GetImportTask().GetTaskId()
|
2023-07-11 15:18:28 +08:00
|
|
|
return func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64,
|
2023-09-21 09:45:27 +08:00
|
|
|
targetChName string, rowCount int64, partID int64,
|
|
|
|
) error {
|
2023-07-11 15:18:28 +08:00
|
|
|
logFields := []zap.Field{
|
2022-12-12 10:57:22 +08:00
|
|
|
zap.Int64("task ID", importTaskID),
|
2023-07-14 15:56:31 +08:00
|
|
|
zap.Int64("partitionID", partID),
|
|
|
|
zap.Int64("segmentID", segmentID),
|
2023-07-11 15:18:28 +08:00
|
|
|
zap.String("target channel name", targetChName),
|
|
|
|
zap.Int64("row count", rowCount),
|
|
|
|
zap.Uint64("ts", ts),
|
|
|
|
}
|
|
|
|
log.Info("adding segment to the correct DataNode flow graph and saving binlog paths", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
|
|
|
|
err := retry.Do(context.Background(), func() error {
|
|
|
|
// Ask DataCoord to save binlog path and add segment to the corresponding DataNode flow graph.
|
2023-10-13 09:55:34 +08:00
|
|
|
err := node.broker.SaveImportSegment(context.Background(), &datapb.SaveImportSegmentRequest{
|
2022-12-12 10:57:22 +08:00
|
|
|
Base: commonpbutil.NewMsgBase(
|
|
|
|
commonpbutil.WithTimeStamp(ts), // Pass current timestamp downstream.
|
|
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
|
|
),
|
|
|
|
SegmentId: segmentID,
|
|
|
|
ChannelName: targetChName,
|
|
|
|
CollectionId: req.GetImportTask().GetCollectionId(),
|
2023-07-11 15:18:28 +08:00
|
|
|
PartitionId: partID,
|
2022-12-12 10:57:22 +08:00
|
|
|
RowNum: rowCount,
|
|
|
|
SaveBinlogPathReq: &datapb.SaveBinlogPathsRequest{
|
|
|
|
Base: commonpbutil.NewMsgBase(
|
|
|
|
commonpbutil.WithTimeStamp(ts),
|
2023-07-11 15:18:28 +08:00
|
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
2022-12-12 10:57:22 +08:00
|
|
|
),
|
|
|
|
SegmentID: segmentID,
|
|
|
|
CollectionID: req.GetImportTask().GetCollectionId(),
|
|
|
|
Field2BinlogPaths: fieldsInsert,
|
|
|
|
Field2StatslogPaths: fieldsStats,
|
|
|
|
// Set start positions of a SaveBinlogPathRequest explicitly.
|
|
|
|
StartPositions: []*datapb.SegmentStartPosition{
|
|
|
|
{
|
2023-03-04 23:21:50 +08:00
|
|
|
StartPosition: &msgpb.MsgPosition{
|
2022-12-12 10:57:22 +08:00
|
|
|
ChannelName: targetChName,
|
|
|
|
Timestamp: ts,
|
|
|
|
},
|
|
|
|
SegmentID: segmentID,
|
|
|
|
},
|
|
|
|
},
|
2024-01-22 14:36:55 +08:00
|
|
|
CheckPoints: []*datapb.CheckPoint{
|
|
|
|
{
|
|
|
|
SegmentID: segmentID,
|
|
|
|
Position: &msgpb.MsgPosition{
|
|
|
|
ChannelName: targetChName,
|
|
|
|
Timestamp: ts,
|
|
|
|
},
|
|
|
|
NumOfRows: rowCount,
|
|
|
|
},
|
|
|
|
},
|
2022-12-12 10:57:22 +08:00
|
|
|
Importing: true,
|
|
|
|
},
|
|
|
|
})
|
|
|
|
// Only retrying when DataCoord is unhealthy or err != nil, otherwise return immediately.
|
|
|
|
if err != nil {
|
2023-10-13 09:55:34 +08:00
|
|
|
if errors.Is(err, merr.ErrServiceNotReady) {
|
|
|
|
return retry.Unrecoverable(err)
|
|
|
|
}
|
|
|
|
return err
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
return nil
|
2024-01-22 14:36:55 +08:00
|
|
|
}, retry.Attempts(60)) // about 3min
|
2022-12-12 10:57:22 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to save import segment", zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
2023-07-11 15:18:28 +08:00
|
|
|
log.Info("segment imported and persisted", logFields...)
|
2022-12-12 10:57:22 +08:00
|
|
|
res.Segments = append(res.Segments, segmentID)
|
|
|
|
res.RowCount += rowCount
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func composeAssignSegmentIDRequest(rowNum int, shardID int, chNames []string,
|
2023-09-21 09:45:27 +08:00
|
|
|
collID int64, partID int64,
|
|
|
|
) *datapb.AssignSegmentIDRequest {
|
2022-12-12 10:57:22 +08:00
|
|
|
// use the first field's row count as segment row count
|
|
|
|
// all the fields row count are same, checked by ImportWrapper
|
|
|
|
// ask DataCoord to alloc a new segment
|
|
|
|
segReqs := []*datapb.SegmentIDRequest{
|
|
|
|
{
|
|
|
|
ChannelName: chNames[shardID],
|
|
|
|
Count: uint32(rowNum),
|
|
|
|
CollectionID: collID,
|
|
|
|
PartitionID: partID,
|
|
|
|
IsImport: true,
|
|
|
|
},
|
|
|
|
}
|
|
|
|
segmentIDReq := &datapb.AssignSegmentIDRequest{
|
|
|
|
NodeID: 0,
|
|
|
|
PeerRole: typeutil.ProxyRole,
|
|
|
|
SegmentIDRequests: segReqs,
|
|
|
|
}
|
|
|
|
return segmentIDReq
|
|
|
|
}
|
|
|
|
|
|
|
|
func createBinLogs(rowNum int, schema *schemapb.CollectionSchema, ts Timestamp,
|
2023-09-21 09:45:27 +08:00
|
|
|
fields map[storage.FieldID]storage.FieldData, node *DataNode, segmentID, colID, partID UniqueID,
|
|
|
|
) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
|
2022-12-12 10:57:22 +08:00
|
|
|
ctx, cancel := context.WithCancel(context.Background())
|
|
|
|
defer cancel()
|
|
|
|
|
|
|
|
tsFieldData := make([]int64, rowNum)
|
|
|
|
for i := range tsFieldData {
|
|
|
|
tsFieldData[i] = int64(ts)
|
|
|
|
}
|
|
|
|
fields[common.TimeStampField] = &storage.Int64FieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: tsFieldData,
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-10-13 09:55:34 +08:00
|
|
|
if err := node.broker.UpdateSegmentStatistics(context.TODO(), &datapb.UpdateSegmentStatisticsRequest{
|
|
|
|
Stats: []*commonpb.SegmentStats{
|
|
|
|
{
|
|
|
|
SegmentID: segmentID,
|
|
|
|
NumRows: int64(rowNum),
|
|
|
|
},
|
|
|
|
},
|
|
|
|
}); err != nil {
|
|
|
|
return nil, nil, err
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
2023-11-15 15:24:18 +08:00
|
|
|
insertData := &InsertData{
|
2022-12-12 10:57:22 +08:00
|
|
|
Data: fields,
|
2023-11-15 15:24:18 +08:00
|
|
|
}
|
|
|
|
// data.updateSize(int64(rowNum))
|
2022-12-12 10:57:22 +08:00
|
|
|
meta := &etcdpb.CollectionMeta{
|
|
|
|
ID: colID,
|
|
|
|
Schema: schema,
|
|
|
|
}
|
2023-05-29 10:21:28 +08:00
|
|
|
iCodec := storage.NewInsertCodecWithSchema(meta)
|
|
|
|
|
2023-11-15 15:24:18 +08:00
|
|
|
binLogs, err := iCodec.Serialize(partID, segmentID, insertData)
|
2022-12-12 10:57:22 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
|
2023-03-23 19:43:57 +08:00
|
|
|
start, _, err := node.allocator.Alloc(uint32(len(binLogs)))
|
2022-12-12 10:57:22 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
field2Insert := make(map[UniqueID]*datapb.Binlog, len(binLogs))
|
|
|
|
kvs := make(map[string][]byte, len(binLogs))
|
|
|
|
field2Logidx := make(map[UniqueID]UniqueID, len(binLogs))
|
|
|
|
for idx, blob := range binLogs {
|
|
|
|
fieldID, err := strconv.ParseInt(blob.GetKey(), 10, 64)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("Flush failed ... cannot parse string to fieldID ..", zap.Error(err))
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
logidx := start + int64(idx)
|
|
|
|
|
|
|
|
k := metautil.JoinIDPath(colID, partID, segmentID, fieldID, logidx)
|
|
|
|
|
|
|
|
key := path.Join(node.chunkManager.RootPath(), common.SegmentInsertLogPath, k)
|
|
|
|
kvs[key] = blob.Value[:]
|
|
|
|
field2Insert[fieldID] = &datapb.Binlog{
|
2023-11-15 15:24:18 +08:00
|
|
|
EntriesNum: int64(rowNum),
|
2022-12-12 10:57:22 +08:00
|
|
|
TimestampFrom: ts,
|
|
|
|
TimestampTo: ts,
|
|
|
|
LogPath: key,
|
|
|
|
LogSize: int64(len(blob.Value)),
|
|
|
|
}
|
|
|
|
field2Logidx[fieldID] = logidx
|
|
|
|
}
|
|
|
|
|
|
|
|
field2Stats := make(map[UniqueID]*datapb.Binlog)
|
|
|
|
// write stats binlog
|
2023-11-15 15:24:18 +08:00
|
|
|
statsBinLog, err := iCodec.SerializePkStatsByData(insertData)
|
2023-05-29 10:21:28 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
fieldID, err := strconv.ParseInt(statsBinLog.GetKey(), 10, 64)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("Flush failed ... cannot parse string to fieldID ..", zap.Error(err))
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
2022-12-12 10:57:22 +08:00
|
|
|
|
2023-05-29 10:21:28 +08:00
|
|
|
logidx := field2Logidx[fieldID]
|
2022-12-12 10:57:22 +08:00
|
|
|
|
2023-05-29 10:21:28 +08:00
|
|
|
// no error raise if alloc=false
|
|
|
|
k := metautil.JoinIDPath(colID, partID, segmentID, fieldID, logidx)
|
2022-12-12 10:57:22 +08:00
|
|
|
|
2023-05-29 10:21:28 +08:00
|
|
|
key := path.Join(node.chunkManager.RootPath(), common.SegmentStatslogPath, k)
|
|
|
|
kvs[key] = statsBinLog.Value
|
|
|
|
field2Stats[fieldID] = &datapb.Binlog{
|
2023-11-15 15:24:18 +08:00
|
|
|
EntriesNum: int64(rowNum),
|
2023-05-29 10:21:28 +08:00
|
|
|
TimestampFrom: ts,
|
|
|
|
TimestampTo: ts,
|
|
|
|
LogPath: key,
|
|
|
|
LogSize: int64(len(statsBinLog.Value)),
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
err = node.chunkManager.MultiWrite(ctx, kvs)
|
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
var (
|
|
|
|
fieldInsert []*datapb.FieldBinlog
|
|
|
|
fieldStats []*datapb.FieldBinlog
|
|
|
|
)
|
|
|
|
for k, v := range field2Insert {
|
|
|
|
fieldInsert = append(fieldInsert, &datapb.FieldBinlog{FieldID: k, Binlogs: []*datapb.Binlog{v}})
|
|
|
|
}
|
|
|
|
for k, v := range field2Stats {
|
|
|
|
fieldStats = append(fieldStats, &datapb.FieldBinlog{FieldID: k, Binlogs: []*datapb.Binlog{v}})
|
|
|
|
}
|
|
|
|
return fieldInsert, fieldStats, nil
|
|
|
|
}
|
|
|
|
|
2023-07-11 15:18:28 +08:00
|
|
|
func reportImportFunc(node *DataNode) importutil.ReportFunc {
|
|
|
|
return func(importResult *rootcoordpb.ImportResult) error {
|
|
|
|
err := retry.Do(context.Background(), func() error {
|
2023-10-13 09:55:34 +08:00
|
|
|
err := node.broker.ReportImport(context.Background(), importResult)
|
2023-07-11 15:18:28 +08:00
|
|
|
if err != nil {
|
2023-10-13 09:55:34 +08:00
|
|
|
log.Error("failed to report import state to RootCoord", zap.Error(err))
|
2023-07-11 15:18:28 +08:00
|
|
|
}
|
2023-10-13 09:55:34 +08:00
|
|
|
return err
|
2023-07-11 15:18:28 +08:00
|
|
|
}, retry.Attempts(node.reportImportRetryTimes))
|
|
|
|
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-12-12 10:57:22 +08:00
|
|
|
func logDupFlush(cID, segID int64) {
|
|
|
|
log.Info("segment is already being flushed, ignoring flush request",
|
2023-07-14 15:56:31 +08:00
|
|
|
zap.Int64("collectionID", cID),
|
|
|
|
zap.Int64("segmentID", segID))
|
2022-12-12 10:57:22 +08:00
|
|
|
}
|
2023-12-04 19:56:35 +08:00
|
|
|
|
|
|
|
func (node *DataNode) PreImport(ctx context.Context, req *datapb.PreImportRequest) (*commonpb.Status, error) {
|
2024-01-31 20:45:04 +08:00
|
|
|
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
|
|
|
zap.Int64("jobID", req.GetJobID()),
|
|
|
|
zap.Int64("collectionID", req.GetCollectionID()),
|
|
|
|
zap.Int64s("partitionIDs", req.GetPartitionIDs()),
|
|
|
|
zap.Strings("vchannels", req.GetVchannels()),
|
|
|
|
zap.Any("files", req.GetImportFiles()))
|
|
|
|
|
|
|
|
log.Info("datanode receive preimport request")
|
|
|
|
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
|
|
return merr.Status(err), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
task := importv2.NewPreImportTask(req)
|
|
|
|
node.importManager.Add(task)
|
|
|
|
|
|
|
|
log.Info("datanode added preimport task")
|
|
|
|
return merr.Success(), nil
|
2023-12-04 19:56:35 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (node *DataNode) ImportV2(ctx context.Context, req *datapb.ImportRequest) (*commonpb.Status, error) {
|
2024-01-31 20:45:04 +08:00
|
|
|
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
|
|
|
zap.Int64("jobID", req.GetJobID()),
|
|
|
|
zap.Int64("collectionID", req.GetCollectionID()),
|
|
|
|
zap.Any("segments", req.GetRequestSegments()),
|
|
|
|
zap.Any("files", req.GetFiles()))
|
|
|
|
|
|
|
|
log.Info("datanode receive import request")
|
|
|
|
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
|
|
return merr.Status(err), nil
|
|
|
|
}
|
|
|
|
task := importv2.NewImportTask(req)
|
|
|
|
node.importManager.Add(task)
|
|
|
|
|
|
|
|
log.Info("datanode added import task")
|
|
|
|
return merr.Success(), nil
|
2023-12-04 19:56:35 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (node *DataNode) QueryPreImport(ctx context.Context, req *datapb.QueryPreImportRequest) (*datapb.QueryPreImportResponse, error) {
|
2024-01-31 20:45:04 +08:00
|
|
|
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
|
|
|
zap.Int64("jobID", req.GetJobID()))
|
|
|
|
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
|
|
return &datapb.QueryPreImportResponse{Status: merr.Status(err)}, nil
|
|
|
|
}
|
|
|
|
status := merr.Success()
|
|
|
|
task := node.importManager.Get(req.GetTaskID())
|
|
|
|
if task == nil || task.GetType() != importv2.PreImportTaskType {
|
|
|
|
status = merr.Status(importv2.WrapNoTaskError(req.GetTaskID(), importv2.PreImportTaskType))
|
|
|
|
}
|
|
|
|
log.RatedInfo(10, "datanode query preimport done", zap.String("state", task.GetState().String()),
|
|
|
|
zap.String("reason", task.GetReason()))
|
|
|
|
return &datapb.QueryPreImportResponse{
|
|
|
|
Status: status,
|
|
|
|
TaskID: task.GetTaskID(),
|
|
|
|
State: task.GetState(),
|
|
|
|
Reason: task.GetReason(),
|
|
|
|
FileStats: task.(*importv2.PreImportTask).GetFileStats(),
|
|
|
|
}, nil
|
2023-12-04 19:56:35 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (node *DataNode) QueryImport(ctx context.Context, req *datapb.QueryImportRequest) (*datapb.QueryImportResponse, error) {
|
2024-01-31 20:45:04 +08:00
|
|
|
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
|
|
|
zap.Int64("jobID", req.GetJobID()))
|
|
|
|
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
|
|
return &datapb.QueryImportResponse{Status: merr.Status(err)}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
status := merr.Success()
|
|
|
|
|
|
|
|
// query slot
|
|
|
|
if req.GetQuerySlot() {
|
|
|
|
return &datapb.QueryImportResponse{
|
|
|
|
Status: status,
|
|
|
|
Slots: node.importManager.Slots(),
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// query import
|
|
|
|
task := node.importManager.Get(req.GetTaskID())
|
|
|
|
if task == nil || task.GetType() != importv2.ImportTaskType {
|
|
|
|
status = merr.Status(importv2.WrapNoTaskError(req.GetTaskID(), importv2.ImportTaskType))
|
|
|
|
}
|
|
|
|
log.RatedInfo(10, "datanode query import done", zap.String("state", task.GetState().String()),
|
|
|
|
zap.String("reason", task.GetReason()))
|
|
|
|
return &datapb.QueryImportResponse{
|
|
|
|
Status: status,
|
|
|
|
TaskID: task.GetTaskID(),
|
|
|
|
State: task.GetState(),
|
|
|
|
Reason: task.GetReason(),
|
|
|
|
ImportSegmentsInfo: task.(*importv2.ImportTask).GetSegmentsInfo(),
|
|
|
|
}, nil
|
2023-12-04 19:56:35 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (node *DataNode) DropImport(ctx context.Context, req *datapb.DropImportRequest) (*commonpb.Status, error) {
|
2024-01-31 20:45:04 +08:00
|
|
|
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
|
|
|
zap.Int64("jobID", req.GetJobID()))
|
|
|
|
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
|
|
return merr.Status(err), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
node.importManager.Remove(req.GetTaskID())
|
|
|
|
|
|
|
|
log.Info("datanode drop import done")
|
|
|
|
|
|
|
|
return merr.Success(), nil
|
2023-12-04 19:56:35 +08:00
|
|
|
}
|