feat: back source error detail (#1437)
Signed-off-by: Jim Ma <majinjing3@gmail.com>
This commit is contained in:
parent
b3884178fb
commit
cf7debf9c4
|
|
@ -30,6 +30,7 @@ import (
|
|||
"go.opentelemetry.io/otel/trace"
|
||||
"go.uber.org/atomic"
|
||||
"golang.org/x/time/rate"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
"d7y.io/dragonfly/v2/client/config"
|
||||
"d7y.io/dragonfly/v2/client/daemon/metrics"
|
||||
|
|
@ -162,6 +163,8 @@ type peerTaskConductor struct {
|
|||
// subtask only
|
||||
parent *peerTaskConductor
|
||||
rg *util.Range
|
||||
|
||||
sourceErrorStatus *status.Status
|
||||
}
|
||||
|
||||
func (ptm *peerTaskManager) newPeerTaskConductor(
|
||||
|
|
@ -440,6 +443,10 @@ func (pt *peerTaskConductor) Log() *logger.SugaredLoggerOnWith {
|
|||
return pt.SugaredLoggerOnWith
|
||||
}
|
||||
|
||||
func (pt *peerTaskConductor) UpdateSourceErrorStatus(st *status.Status) {
|
||||
pt.sourceErrorStatus = st
|
||||
}
|
||||
|
||||
func (pt *peerTaskConductor) cancel(code base.Code, reason string) {
|
||||
pt.statusOnce.Do(func() {
|
||||
pt.failedCode = code
|
||||
|
|
@ -1635,3 +1642,10 @@ func (pt *peerTaskConductor) sendPieceResult(pr *scheduler.PieceResult) error {
|
|||
pt.sendPieceResultLock.Unlock()
|
||||
return err
|
||||
}
|
||||
|
||||
func (pt *peerTaskConductor) getFailedError() error {
|
||||
if pt.sourceErrorStatus != nil {
|
||||
return pt.sourceErrorStatus.Err()
|
||||
}
|
||||
return fmt.Errorf("peer task failed: %d/%s", pt.failedCode, pt.failedReason)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -30,6 +30,7 @@ import (
|
|||
"go.opentelemetry.io/otel"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
"golang.org/x/time/rate"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
"d7y.io/dragonfly/v2/client/config"
|
||||
"d7y.io/dragonfly/v2/client/daemon/metrics"
|
||||
|
|
@ -97,6 +98,8 @@ type Task interface {
|
|||
|
||||
PublishPieceInfo(pieceNum int32, size uint32)
|
||||
ReportPieceResult(request *DownloadPieceRequest, result *DownloadPieceResult, err error)
|
||||
|
||||
UpdateSourceErrorStatus(st *status.Status)
|
||||
}
|
||||
|
||||
type Logger interface {
|
||||
|
|
@ -366,6 +369,7 @@ type SubscribeResponse struct {
|
|||
PieceInfoChannel chan *PieceInfo
|
||||
Success chan struct{}
|
||||
Fail chan struct{}
|
||||
FailReason func() error
|
||||
}
|
||||
|
||||
func (ptm *peerTaskManager) Subscribe(request *base.PieceTaskRequest) (*SubscribeResponse, bool) {
|
||||
|
|
@ -379,6 +383,7 @@ func (ptm *peerTaskManager) Subscribe(request *base.PieceTaskRequest) (*Subscrib
|
|||
PieceInfoChannel: ptc.broker.Subscribe(),
|
||||
Success: ptc.successCh,
|
||||
Fail: ptc.failCh,
|
||||
FailReason: ptc.getFailedError,
|
||||
}
|
||||
return result, true
|
||||
}
|
||||
|
|
|
|||
|
|
@ -14,6 +14,7 @@ import (
|
|||
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
scheduler "d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
||||
gomock "github.com/golang/mock/gomock"
|
||||
status "google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
// MockTaskManager is a mock of TaskManager interface.
|
||||
|
|
@ -395,6 +396,18 @@ func (mr *MockTaskMockRecorder) SetTotalPieces(arg0 interface{}) *gomock.Call {
|
|||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetTotalPieces", reflect.TypeOf((*MockTask)(nil).SetTotalPieces), arg0)
|
||||
}
|
||||
|
||||
// UpdateSourceErrorStatus mocks base method.
|
||||
func (m *MockTask) UpdateSourceErrorStatus(st *status.Status) {
|
||||
m.ctrl.T.Helper()
|
||||
m.ctrl.Call(m, "UpdateSourceErrorStatus", st)
|
||||
}
|
||||
|
||||
// UpdateSourceErrorStatus indicates an expected call of UpdateSourceErrorStatus.
|
||||
func (mr *MockTaskMockRecorder) UpdateSourceErrorStatus(st interface{}) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateSourceErrorStatus", reflect.TypeOf((*MockTask)(nil).UpdateSourceErrorStatus), st)
|
||||
}
|
||||
|
||||
// MockLogger is a mock of Logger interface.
|
||||
type MockLogger struct {
|
||||
ctrl *gomock.Controller
|
||||
|
|
|
|||
|
|
@ -360,6 +360,9 @@ func (ptm *peerTaskManager) tryReuseSeedPeerTask(ctx context.Context,
|
|||
PieceInfoChannel: nil,
|
||||
Success: successCh,
|
||||
Fail: nil,
|
||||
FailReason: func() error {
|
||||
return nil
|
||||
},
|
||||
},
|
||||
}, true
|
||||
}
|
||||
|
|
|
|||
|
|
@ -79,6 +79,7 @@ func (ptm *peerTaskManager) newSeedTask(
|
|||
PieceInfoChannel: ptc.broker.Subscribe(),
|
||||
Success: ptc.successCh,
|
||||
Fail: ptc.failCh,
|
||||
FailReason: ptc.getFailedError,
|
||||
},
|
||||
}
|
||||
return resp, nil
|
||||
|
|
|
|||
|
|
@ -111,9 +111,8 @@ func (s *streamTask) Start(ctx context.Context) (io.ReadCloser, map[string]strin
|
|||
s.span.End()
|
||||
return nil, attr, ctx.Err()
|
||||
case <-s.peerTaskConductor.failCh:
|
||||
err := fmt.Errorf("peer task failed: %d/%s",
|
||||
s.peerTaskConductor.failedCode, s.peerTaskConductor.failedReason)
|
||||
s.Errorf("wait first piece failed due to %s ", err.Error())
|
||||
err := s.peerTaskConductor.getFailedError()
|
||||
s.Errorf("wait first piece failed due to %s", err.Error())
|
||||
return nil, attr, err
|
||||
case <-s.peerTaskConductor.successCh:
|
||||
if s.peerTaskConductor.GetContentLength() != -1 {
|
||||
|
|
|
|||
|
|
@ -27,6 +27,8 @@ import (
|
|||
"net/url"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
"d7y.io/dragonfly/v2/client/daemon/storage"
|
||||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||
"d7y.io/dragonfly/v2/pkg/digest"
|
||||
|
|
@ -73,6 +75,7 @@ type pieceDownloadError struct {
|
|||
|
||||
type backSourceError struct {
|
||||
err error
|
||||
st *status.Status
|
||||
}
|
||||
|
||||
func isConnectionError(err error) bool {
|
||||
|
|
@ -107,6 +110,9 @@ func (e *pieceDownloadError) Error() string {
|
|||
}
|
||||
|
||||
func (e *backSourceError) Error() string {
|
||||
if e.st != nil {
|
||||
return e.st.Err().Error()
|
||||
}
|
||||
return e.err.Error()
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -31,6 +31,8 @@ import (
|
|||
"github.com/go-http-utils/headers"
|
||||
"go.uber.org/atomic"
|
||||
"golang.org/x/time/rate"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
"d7y.io/dragonfly/v2/client/config"
|
||||
"d7y.io/dragonfly/v2/client/daemon/storage"
|
||||
|
|
@ -41,6 +43,7 @@ import (
|
|||
"d7y.io/dragonfly/v2/pkg/retry"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/dfdaemon"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/errordetails"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/scheduler"
|
||||
"d7y.io/dragonfly/v2/pkg/source"
|
||||
)
|
||||
|
|
@ -294,7 +297,10 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, peerTaskReq
|
|||
// 1. support range request
|
||||
// 2. target content length is greater than concurrentOption.ThresholdSize
|
||||
metadata, err := source.GetMetadata(backSourceRequest)
|
||||
if err == nil && metadata.SupportRange && metadata.TotalContentLength > -1 {
|
||||
if err == nil && metadata.Validate != nil && metadata.Validate() == nil {
|
||||
if !metadata.SupportRange || metadata.TotalContentLength == -1 {
|
||||
goto singleDownload
|
||||
}
|
||||
var (
|
||||
targetContentLength int64
|
||||
)
|
||||
|
|
@ -336,6 +342,7 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, peerTaskReq
|
|||
}
|
||||
}
|
||||
|
||||
singleDownload:
|
||||
// 1. download pieces from source
|
||||
response, err := source.Download(backSourceRequest)
|
||||
// TODO update expire info
|
||||
|
|
@ -344,8 +351,34 @@ func (pm *pieceManager) DownloadSource(ctx context.Context, pt Task, peerTaskReq
|
|||
}
|
||||
err = response.Validate()
|
||||
if err != nil {
|
||||
log.Errorf("back source status code %s", response.Status)
|
||||
// convert error details to status
|
||||
st := status.Newf(codes.Aborted, "response is not valid")
|
||||
hdr := map[string]string{}
|
||||
for k, v := range response.Header {
|
||||
if len(v) > 0 {
|
||||
hdr[k] = response.Header.Get(k)
|
||||
}
|
||||
}
|
||||
srcErr := &errordetails.SourceError{
|
||||
Temporary: response.Temporary != nil && response.Temporary(),
|
||||
Metadata: &base.ExtendAttribute{
|
||||
Header: hdr,
|
||||
StatusCode: int32(response.StatusCode),
|
||||
Status: response.Status,
|
||||
},
|
||||
}
|
||||
st, err = st.WithDetails(srcErr)
|
||||
if err != nil {
|
||||
log.Errorf("convert source error details error: %s", err.Error())
|
||||
return err
|
||||
}
|
||||
pt.UpdateSourceErrorStatus(st)
|
||||
return &backSourceError{
|
||||
err: st.Err(),
|
||||
st: st,
|
||||
}
|
||||
}
|
||||
contentLength := response.ContentLength
|
||||
if contentLength < 0 {
|
||||
log.Warnf("can not get content length for %s", peerTaskRequest.Url)
|
||||
|
|
|
|||
|
|
@ -180,10 +180,15 @@ func (s *seedSynchronizer) sendPieceSeeds(reuse bool) (err error) {
|
|||
}
|
||||
return err
|
||||
case <-s.Fail:
|
||||
s.Error("seed task failed")
|
||||
reason := s.FailReason()
|
||||
s.Errorf("seed task failed: %s", reason)
|
||||
s.Span.RecordError(err)
|
||||
s.Span.SetAttributes(config.AttributeSeedTaskSuccess.Bool(false))
|
||||
return status.Errorf(codes.Internal, "seed task failed")
|
||||
// return underlay status to scheduler
|
||||
if st, ok := status.FromError(reason); ok {
|
||||
return st.Err()
|
||||
}
|
||||
return status.Errorf(codes.Internal, "seed task failed: %s", reason)
|
||||
case p := <-s.PieceInfoChannel:
|
||||
s.Infof("receive piece info, num: %d, ordered num: %d, finish: %v", p.Num, p.OrderedNum, p.Finished)
|
||||
contentLength, desired, err = s.sendOrderedPieceSeeds(desired, p.OrderedNum, p.Finished)
|
||||
|
|
|
|||
|
|
@ -230,8 +230,13 @@ loop:
|
|||
s.Unlock()
|
||||
break loop
|
||||
case <-s.Fail:
|
||||
s.Errorf("peer task failed")
|
||||
return dferrors.Newf(base.Code_ClientError, "peer task failed")
|
||||
reason := s.FailReason()
|
||||
s.Errorf("peer task failed: %s", reason)
|
||||
// return underlay status to peer
|
||||
if st, ok := status.FromError(reason); ok {
|
||||
return st.Err()
|
||||
}
|
||||
return status.Errorf(codes.Internal, "peer task failed: %s", reason)
|
||||
}
|
||||
}
|
||||
select {
|
||||
|
|
|
|||
|
|
@ -33,6 +33,7 @@ import (
|
|||
|
||||
"github.com/go-http-utils/headers"
|
||||
"go.opentelemetry.io/otel/propagation"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
"d7y.io/dragonfly/v2/client/config"
|
||||
"d7y.io/dragonfly/v2/client/daemon/metrics"
|
||||
|
|
@ -41,6 +42,7 @@ import (
|
|||
logger "d7y.io/dragonfly/v2/internal/dflog"
|
||||
nethttp "d7y.io/dragonfly/v2/pkg/net/http"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
"d7y.io/dragonfly/v2/pkg/rpc/errordetails"
|
||||
)
|
||||
|
||||
var _ *logger.SugaredLoggerOnWith // pin this package for no log code generation
|
||||
|
|
@ -254,7 +256,29 @@ func (rt *transport) download(ctx context.Context, req *http.Request) (*http.Res
|
|||
},
|
||||
)
|
||||
if err != nil {
|
||||
log.Errorf("download fail: %v", err)
|
||||
log.Errorf("start steam task error: %v", err)
|
||||
// check underlay status code
|
||||
if st, ok := status.FromError(err); ok {
|
||||
for _, detail := range st.Details() {
|
||||
switch d := detail.(type) {
|
||||
case *errordetails.SourceError:
|
||||
hdr := nethttp.MapToHeader(attr)
|
||||
for k, v := range d.Metadata.Header {
|
||||
hdr.Set(k, v)
|
||||
}
|
||||
resp := &http.Response{
|
||||
StatusCode: int(d.Metadata.StatusCode),
|
||||
Body: io.NopCloser(bytes.NewBufferString(d.Metadata.Status)),
|
||||
Header: hdr,
|
||||
Proto: req.Proto,
|
||||
ProtoMajor: req.ProtoMajor,
|
||||
ProtoMinor: req.ProtoMinor,
|
||||
}
|
||||
log.Errorf("underlay response code: %d", d.Metadata.StatusCode)
|
||||
return resp, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
// add more info for debugging
|
||||
if attr != nil {
|
||||
err = fmt.Errorf("task: %s\npeer: %s\nerror: %s",
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@ PROTOC_ALL_IMAGE=${PROTOC_ALL_IMAGE:-"namely/protoc-all:1.47_0"}
|
|||
PROTO_PATH=pkg/rpc
|
||||
LANGUAGE=go
|
||||
|
||||
proto_modules="base cdnsystem dfdaemon manager scheduler"
|
||||
proto_modules="base cdnsystem dfdaemon manager scheduler errordetails"
|
||||
|
||||
echo "generate protos..."
|
||||
|
||||
|
|
|
|||
|
|
@ -763,6 +763,10 @@ type ExtendAttribute struct {
|
|||
|
||||
// task response header, eg: HTTP Response Header
|
||||
Header map[string]string `protobuf:"bytes,1,rep,name=header,proto3" json:"header,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
|
||||
// task response code, eg: HTTP Status Code
|
||||
StatusCode int32 `protobuf:"varint,2,opt,name=status_code,json=statusCode,proto3" json:"status_code,omitempty"`
|
||||
// task response status, eg: HTTP Status
|
||||
Status string `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
|
||||
}
|
||||
|
||||
func (x *ExtendAttribute) Reset() {
|
||||
|
|
@ -804,6 +808,20 @@ func (x *ExtendAttribute) GetHeader() map[string]string {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (x *ExtendAttribute) GetStatusCode() int32 {
|
||||
if x != nil {
|
||||
return x.StatusCode
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *ExtendAttribute) GetStatus() string {
|
||||
if x != nil {
|
||||
return x.Status
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type PiecePacket struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
|
|
@ -984,92 +1002,95 @@ var file_pkg_rpc_base_base_proto_rawDesc = []byte{
|
|||
0x65, 0x53, 0x74, 0x79, 0x6c, 0x65, 0x12, 0x2c, 0x0a, 0x0d, 0x64, 0x6f, 0x77, 0x6e, 0x6c, 0x6f,
|
||||
0x61, 0x64, 0x5f, 0x63, 0x6f, 0x73, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x42, 0x07, 0xfa,
|
||||
0x42, 0x04, 0x32, 0x02, 0x28, 0x00, 0x52, 0x0c, 0x64, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64,
|
||||
0x43, 0x6f, 0x73, 0x74, 0x22, 0x87, 0x01, 0x0a, 0x0f, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x41,
|
||||
0x43, 0x6f, 0x73, 0x74, 0x22, 0xc0, 0x01, 0x0a, 0x0f, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x41,
|
||||
0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x39, 0x0a, 0x06, 0x68, 0x65, 0x61, 0x64,
|
||||
0x65, 0x72, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e,
|
||||
0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x2e,
|
||||
0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x68, 0x65, 0x61,
|
||||
0x64, 0x65, 0x72, 0x1a, 0x39, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74,
|
||||
0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd7,
|
||||
0x02, 0x0a, 0x0b, 0x50, 0x69, 0x65, 0x63, 0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x20,
|
||||
0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42,
|
||||
0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64,
|
||||
0x12, 0x20, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28,
|
||||
0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50,
|
||||
0x69, 0x64, 0x12, 0x22, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x04,
|
||||
0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x07, 0x64,
|
||||
0x73, 0x74, 0x41, 0x64, 0x64, 0x72, 0x12, 0x30, 0x0a, 0x0b, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f,
|
||||
0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61,
|
||||
0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x70, 0x69,
|
||||
0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61,
|
||||
0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x74,
|
||||
0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e,
|
||||
0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28,
|
||||
0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68,
|
||||
0x12, 0x24, 0x0a, 0x0e, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x6d, 0x64, 0x35, 0x5f, 0x73, 0x69,
|
||||
0x67, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4d,
|
||||
0x64, 0x35, 0x53, 0x69, 0x67, 0x6e, 0x12, 0x40, 0x0a, 0x10, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x64,
|
||||
0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x15, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x41, 0x74,
|
||||
0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x52, 0x0f, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x41,
|
||||
0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x2a, 0x94, 0x05, 0x0a, 0x04, 0x43, 0x6f, 0x64,
|
||||
0x65, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49,
|
||||
0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x07, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x10,
|
||||
0xc8, 0x01, 0x12, 0x16, 0x0a, 0x11, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x76,
|
||||
0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x10, 0xf4, 0x03, 0x12, 0x13, 0x0a, 0x0e, 0x52, 0x65,
|
||||
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4c, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0xe8, 0x07, 0x12,
|
||||
0x0f, 0x0a, 0x0a, 0x42, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x10, 0xf8, 0x0a,
|
||||
0x12, 0x15, 0x0a, 0x10, 0x50, 0x65, 0x65, 0x72, 0x54, 0x61, 0x73, 0x6b, 0x4e, 0x6f, 0x74, 0x46,
|
||||
0x6f, 0x75, 0x6e, 0x64, 0x10, 0xfc, 0x0a, 0x12, 0x11, 0x0a, 0x0c, 0x55, 0x6e, 0x6b, 0x6e, 0x6f,
|
||||
0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xdc, 0x0b, 0x12, 0x13, 0x0a, 0x0e, 0x52, 0x65,
|
||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x4f, 0x75, 0x74, 0x10, 0xe0, 0x0b, 0x12,
|
||||
0x10, 0x0a, 0x0b, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xa0,
|
||||
0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65,
|
||||
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa1, 0x1f, 0x12, 0x1a,
|
||||
0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65,
|
||||
0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0xa2, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c,
|
||||
0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65,
|
||||
0x6c, 0x65, 0x64, 0x10, 0xa3, 0x1f, 0x12, 0x19, 0x0a, 0x14, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74,
|
||||
0x57, 0x61, 0x69, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x61, 0x64, 0x79, 0x10, 0xa4,
|
||||
0x1f, 0x12, 0x1c, 0x0a, 0x17, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65,
|
||||
0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa5, 0x1f, 0x12,
|
||||
0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x4c, 0x69, 0x6d, 0x69, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa6, 0x1f, 0x12, 0x1a, 0x0a, 0x15,
|
||||
0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xa7, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65,
|
||||
0x6e, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x72, 0x72, 0x6f,
|
||||
0x72, 0x10, 0xa8, 0x1f, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x69,
|
||||
0x65, 0x63, 0x65, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0xb4, 0x22, 0x12, 0x0f,
|
||||
0x0a, 0x0a, 0x53, 0x63, 0x68, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x88, 0x27, 0x12,
|
||||
0x18, 0x0a, 0x13, 0x53, 0x63, 0x68, 0x65, 0x64, 0x4e, 0x65, 0x65, 0x64, 0x42, 0x61, 0x63, 0x6b,
|
||||
0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x10, 0x89, 0x27, 0x12, 0x12, 0x0a, 0x0d, 0x53, 0x63, 0x68,
|
||||
0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x47, 0x6f, 0x6e, 0x65, 0x10, 0x8a, 0x27, 0x12, 0x16, 0x0a,
|
||||
0x11, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75,
|
||||
0x6e, 0x64, 0x10, 0x8c, 0x27, 0x12, 0x23, 0x0a, 0x1e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65,
|
||||
0x65, 0x72, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x65, 0x70,
|
||||
0x6f, 0x72, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x10, 0x8d, 0x27, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x63,
|
||||
0x68, 0x65, 0x64, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x45, 0x72, 0x72,
|
||||
0x6f, 0x72, 0x10, 0x8e, 0x27, 0x12, 0x18, 0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b,
|
||||
0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x79, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf1, 0x2e, 0x12,
|
||||
0x14, 0x0a, 0x0f, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75,
|
||||
0x6e, 0x64, 0x10, 0x84, 0x32, 0x12, 0x18, 0x0a, 0x13, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64,
|
||||
0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x10, 0xd9, 0x36, 0x2a,
|
||||
0x17, 0x0a, 0x0a, 0x50, 0x69, 0x65, 0x63, 0x65, 0x53, 0x74, 0x79, 0x6c, 0x65, 0x12, 0x09, 0x0a,
|
||||
0x05, 0x50, 0x4c, 0x41, 0x49, 0x4e, 0x10, 0x00, 0x2a, 0x2c, 0x0a, 0x09, 0x53, 0x69, 0x7a, 0x65,
|
||||
0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10,
|
||||
0x00, 0x12, 0x09, 0x0a, 0x05, 0x53, 0x4d, 0x41, 0x4c, 0x4c, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04,
|
||||
0x54, 0x49, 0x4e, 0x59, 0x10, 0x02, 0x2a, 0x2d, 0x0a, 0x07, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72,
|
||||
0x6e, 0x12, 0x07, 0x0a, 0x03, 0x50, 0x32, 0x50, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x45,
|
||||
0x45, 0x44, 0x5f, 0x50, 0x45, 0x45, 0x52, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x4f, 0x55,
|
||||
0x52, 0x43, 0x45, 0x10, 0x02, 0x2a, 0x30, 0x0a, 0x08, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x79, 0x70,
|
||||
0x65, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x10, 0x00, 0x12, 0x0b, 0x0a,
|
||||
0x07, 0x44, 0x66, 0x43, 0x61, 0x63, 0x68, 0x65, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x44, 0x66,
|
||||
0x53, 0x74, 0x6f, 0x72, 0x65, 0x10, 0x02, 0x42, 0x22, 0x5a, 0x20, 0x64, 0x37, 0x79, 0x2e, 0x69,
|
||||
0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70,
|
||||
0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x33,
|
||||
0x64, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x63, 0x6f,
|
||||
0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73,
|
||||
0x43, 0x6f, 0x64, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x1a, 0x39, 0x0a, 0x0b,
|
||||
0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,
|
||||
0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a,
|
||||
0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61,
|
||||
0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd7, 0x02, 0x0a, 0x0b, 0x50, 0x69, 0x65, 0x63,
|
||||
0x65, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x20, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f,
|
||||
0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72, 0x02, 0x10,
|
||||
0x01, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x07, 0x64, 0x73, 0x74,
|
||||
0x5f, 0x70, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa, 0x42, 0x04, 0x72,
|
||||
0x02, 0x10, 0x01, 0x52, 0x06, 0x64, 0x73, 0x74, 0x50, 0x69, 0x64, 0x12, 0x22, 0x0a, 0x08, 0x64,
|
||||
0x73, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x42, 0x07, 0xfa,
|
||||
0x42, 0x04, 0x72, 0x02, 0x10, 0x01, 0x52, 0x07, 0x64, 0x73, 0x74, 0x41, 0x64, 0x64, 0x72, 0x12,
|
||||
0x30, 0x0a, 0x0b, 0x70, 0x69, 0x65, 0x63, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x05,
|
||||
0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x50, 0x69, 0x65, 0x63,
|
||||
0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x70, 0x69, 0x65, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f,
|
||||
0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x70, 0x69, 0x65, 0x63, 0x65,
|
||||
0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x69, 0x65,
|
||||
0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x65,
|
||||
0x6e, 0x67, 0x74, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x74,
|
||||
0x65, 0x6e, 0x74, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x70, 0x69, 0x65,
|
||||
0x63, 0x65, 0x5f, 0x6d, 0x64, 0x35, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x0c, 0x70, 0x69, 0x65, 0x63, 0x65, 0x4d, 0x64, 0x35, 0x53, 0x69, 0x67, 0x6e, 0x12,
|
||||
0x40, 0x0a, 0x10, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62,
|
||||
0x75, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x62, 0x61, 0x73, 0x65,
|
||||
0x2e, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65,
|
||||
0x52, 0x0f, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74,
|
||||
0x65, 0x2a, 0x94, 0x05, 0x0a, 0x04, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x58, 0x5f,
|
||||
0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a,
|
||||
0x07, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x10, 0xc8, 0x01, 0x12, 0x16, 0x0a, 0x11, 0x53,
|
||||
0x65, 0x72, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65,
|
||||
0x10, 0xf4, 0x03, 0x12, 0x13, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4c,
|
||||
0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0xe8, 0x07, 0x12, 0x0f, 0x0a, 0x0a, 0x42, 0x61, 0x64, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x10, 0xf8, 0x0a, 0x12, 0x15, 0x0a, 0x10, 0x50, 0x65, 0x65,
|
||||
0x72, 0x54, 0x61, 0x73, 0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0xfc, 0x0a,
|
||||
0x12, 0x11, 0x0a, 0x0c, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72,
|
||||
0x10, 0xdc, 0x0b, 0x12, 0x13, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69,
|
||||
0x6d, 0x65, 0x4f, 0x75, 0x74, 0x10, 0xe0, 0x0b, 0x12, 0x10, 0x0a, 0x0b, 0x43, 0x6c, 0x69, 0x65,
|
||||
0x6e, 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xa0, 0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c,
|
||||
0x69, 0x65, 0x6e, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x46, 0x61, 0x69, 0x6c, 0x10, 0xa1, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e,
|
||||
0x74, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74,
|
||||
0x10, 0xa2, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e,
|
||||
0x74, 0x65, 0x78, 0x74, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x10, 0xa3, 0x1f, 0x12,
|
||||
0x19, 0x0a, 0x14, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x57, 0x61, 0x69, 0x74, 0x50, 0x69, 0x65,
|
||||
0x63, 0x65, 0x52, 0x65, 0x61, 0x64, 0x79, 0x10, 0xa4, 0x1f, 0x12, 0x1c, 0x0a, 0x17, 0x43, 0x6c,
|
||||
0x69, 0x65, 0x6e, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x6c, 0x6f, 0x61,
|
||||
0x64, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xa5, 0x1f, 0x12, 0x1b, 0x0a, 0x16, 0x43, 0x6c, 0x69, 0x65,
|
||||
0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x46, 0x61,
|
||||
0x69, 0x6c, 0x10, 0xa6, 0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43,
|
||||
0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xa7,
|
||||
0x1f, 0x12, 0x1a, 0x0a, 0x15, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x53,
|
||||
0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0xa8, 0x1f, 0x12, 0x18, 0x0a,
|
||||
0x13, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x69, 0x65, 0x63, 0x65, 0x4e, 0x6f, 0x74, 0x46,
|
||||
0x6f, 0x75, 0x6e, 0x64, 0x10, 0xb4, 0x22, 0x12, 0x0f, 0x0a, 0x0a, 0x53, 0x63, 0x68, 0x65, 0x64,
|
||||
0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x88, 0x27, 0x12, 0x18, 0x0a, 0x13, 0x53, 0x63, 0x68, 0x65,
|
||||
0x64, 0x4e, 0x65, 0x65, 0x64, 0x42, 0x61, 0x63, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x10,
|
||||
0x89, 0x27, 0x12, 0x12, 0x0a, 0x0d, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x47,
|
||||
0x6f, 0x6e, 0x65, 0x10, 0x8a, 0x27, 0x12, 0x16, 0x0a, 0x11, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50,
|
||||
0x65, 0x65, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x8c, 0x27, 0x12, 0x23,
|
||||
0x0a, 0x1e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x50, 0x65, 0x65, 0x72, 0x50, 0x69, 0x65, 0x63, 0x65,
|
||||
0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x61, 0x69, 0x6c,
|
||||
0x10, 0x8d, 0x27, 0x12, 0x19, 0x0a, 0x14, 0x53, 0x63, 0x68, 0x65, 0x64, 0x54, 0x61, 0x73, 0x6b,
|
||||
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x8e, 0x27, 0x12, 0x18,
|
||||
0x0a, 0x13, 0x43, 0x44, 0x4e, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72,
|
||||
0x79, 0x46, 0x61, 0x69, 0x6c, 0x10, 0xf1, 0x2e, 0x12, 0x14, 0x0a, 0x0f, 0x43, 0x44, 0x4e, 0x54,
|
||||
0x61, 0x73, 0x6b, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x84, 0x32, 0x12, 0x18,
|
||||
0x0a, 0x13, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
|
||||
0x65, 0x54, 0x79, 0x70, 0x65, 0x10, 0xd9, 0x36, 0x2a, 0x17, 0x0a, 0x0a, 0x50, 0x69, 0x65, 0x63,
|
||||
0x65, 0x53, 0x74, 0x79, 0x6c, 0x65, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x4c, 0x41, 0x49, 0x4e, 0x10,
|
||||
0x00, 0x2a, 0x2c, 0x0a, 0x09, 0x53, 0x69, 0x7a, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0a,
|
||||
0x0a, 0x06, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x53, 0x4d,
|
||||
0x41, 0x4c, 0x4c, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x54, 0x49, 0x4e, 0x59, 0x10, 0x02, 0x2a,
|
||||
0x2d, 0x0a, 0x07, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x07, 0x0a, 0x03, 0x50, 0x32,
|
||||
0x50, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x45, 0x45, 0x44, 0x5f, 0x50, 0x45, 0x45, 0x52,
|
||||
0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x10, 0x02, 0x2a, 0x30,
|
||||
0x0a, 0x08, 0x54, 0x61, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x6f,
|
||||
0x72, 0x6d, 0x61, 0x6c, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x44, 0x66, 0x43, 0x61, 0x63, 0x68,
|
||||
0x65, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x44, 0x66, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x10, 0x02,
|
||||
0x42, 0x22, 0x5a, 0x20, 0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67, 0x6f,
|
||||
0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f,
|
||||
0x62, 0x61, 0x73, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
|
|
|||
|
|
@ -502,6 +502,10 @@ func (m *ExtendAttribute) Validate() error {
|
|||
|
||||
// no validation rules for Header
|
||||
|
||||
// no validation rules for StatusCode
|
||||
|
||||
// no validation rules for Status
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -162,9 +162,9 @@ message ExtendAttribute{
|
|||
// task response header, eg: HTTP Response Header
|
||||
map<string, string> header = 1;
|
||||
// task response code, eg: HTTP Status Code
|
||||
//int32 code = 2;
|
||||
int32 status_code = 2;
|
||||
// task response status, eg: HTTP Status
|
||||
//string status = 3;
|
||||
string status = 3;
|
||||
}
|
||||
|
||||
message PiecePacket{
|
||||
|
|
|
|||
|
|
@ -0,0 +1,178 @@
|
|||
//
|
||||
// Copyright 2022 The Dragonfly Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.28.0
|
||||
// protoc v3.19.4
|
||||
// source: pkg/rpc/errordetails/error_details.proto
|
||||
|
||||
package errordetails
|
||||
|
||||
import (
|
||||
base "d7y.io/dragonfly/v2/pkg/rpc/base"
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
reflect "reflect"
|
||||
sync "sync"
|
||||
)
|
||||
|
||||
const (
|
||||
// Verify that this generated code is sufficiently up-to-date.
|
||||
_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
|
||||
// Verify that runtime/protoimpl is sufficiently up-to-date.
|
||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
||||
)
|
||||
|
||||
type SourceError struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Temporary bool `protobuf:"varint,1,opt,name=temporary,proto3" json:"temporary,omitempty"`
|
||||
// source response metadata, eg: HTTP Status Code, HTTP Status, HTTP Header
|
||||
Metadata *base.ExtendAttribute `protobuf:"bytes,2,opt,name=metadata,proto3" json:"metadata,omitempty"`
|
||||
}
|
||||
|
||||
func (x *SourceError) Reset() {
|
||||
*x = SourceError{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_pkg_rpc_errordetails_error_details_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *SourceError) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*SourceError) ProtoMessage() {}
|
||||
|
||||
func (x *SourceError) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_pkg_rpc_errordetails_error_details_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use SourceError.ProtoReflect.Descriptor instead.
|
||||
func (*SourceError) Descriptor() ([]byte, []int) {
|
||||
return file_pkg_rpc_errordetails_error_details_proto_rawDescGZIP(), []int{0}
|
||||
}
|
||||
|
||||
func (x *SourceError) GetTemporary() bool {
|
||||
if x != nil {
|
||||
return x.Temporary
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (x *SourceError) GetMetadata() *base.ExtendAttribute {
|
||||
if x != nil {
|
||||
return x.Metadata
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
var File_pkg_rpc_errordetails_error_details_proto protoreflect.FileDescriptor
|
||||
|
||||
var file_pkg_rpc_errordetails_error_details_proto_rawDesc = []byte{
|
||||
0x0a, 0x28, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x64,
|
||||
0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x2f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x64, 0x65, 0x74,
|
||||
0x61, 0x69, 0x6c, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0c, 0x65, 0x72, 0x72, 0x6f,
|
||||
0x72, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x1a, 0x17, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70,
|
||||
0x63, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x2f, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||
0x6f, 0x22, 0x5e, 0x0a, 0x0b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72,
|
||||
0x12, 0x1c, 0x0a, 0x09, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x08, 0x52, 0x09, 0x74, 0x65, 0x6d, 0x70, 0x6f, 0x72, 0x61, 0x72, 0x79, 0x12, 0x31,
|
||||
0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x15, 0x2e, 0x62, 0x61, 0x73, 0x65, 0x2e, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x41, 0x74,
|
||||
0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74,
|
||||
0x61, 0x42, 0x2a, 0x5a, 0x28, 0x64, 0x37, 0x79, 0x2e, 0x69, 0x6f, 0x2f, 0x64, 0x72, 0x61, 0x67,
|
||||
0x6f, 0x6e, 0x66, 0x6c, 0x79, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x72, 0x70, 0x63,
|
||||
0x2f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x62, 0x06, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
file_pkg_rpc_errordetails_error_details_proto_rawDescOnce sync.Once
|
||||
file_pkg_rpc_errordetails_error_details_proto_rawDescData = file_pkg_rpc_errordetails_error_details_proto_rawDesc
|
||||
)
|
||||
|
||||
func file_pkg_rpc_errordetails_error_details_proto_rawDescGZIP() []byte {
|
||||
file_pkg_rpc_errordetails_error_details_proto_rawDescOnce.Do(func() {
|
||||
file_pkg_rpc_errordetails_error_details_proto_rawDescData = protoimpl.X.CompressGZIP(file_pkg_rpc_errordetails_error_details_proto_rawDescData)
|
||||
})
|
||||
return file_pkg_rpc_errordetails_error_details_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_pkg_rpc_errordetails_error_details_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
|
||||
var file_pkg_rpc_errordetails_error_details_proto_goTypes = []any{
|
||||
(*SourceError)(nil), // 0: errordetails.SourceError
|
||||
(*base.ExtendAttribute)(nil), // 1: base.ExtendAttribute
|
||||
}
|
||||
var file_pkg_rpc_errordetails_error_details_proto_depIdxs = []int32{
|
||||
1, // 0: errordetails.SourceError.metadata:type_name -> base.ExtendAttribute
|
||||
1, // [1:1] is the sub-list for method output_type
|
||||
1, // [1:1] is the sub-list for method input_type
|
||||
1, // [1:1] is the sub-list for extension type_name
|
||||
1, // [1:1] is the sub-list for extension extendee
|
||||
0, // [0:1] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_pkg_rpc_errordetails_error_details_proto_init() }
|
||||
func file_pkg_rpc_errordetails_error_details_proto_init() {
|
||||
if File_pkg_rpc_errordetails_error_details_proto != nil {
|
||||
return
|
||||
}
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_pkg_rpc_errordetails_error_details_proto_msgTypes[0].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*SourceError); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
type x struct{}
|
||||
out := protoimpl.TypeBuilder{
|
||||
File: protoimpl.DescBuilder{
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_pkg_rpc_errordetails_error_details_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 1,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
GoTypes: file_pkg_rpc_errordetails_error_details_proto_goTypes,
|
||||
DependencyIndexes: file_pkg_rpc_errordetails_error_details_proto_depIdxs,
|
||||
MessageInfos: file_pkg_rpc_errordetails_error_details_proto_msgTypes,
|
||||
}.Build()
|
||||
File_pkg_rpc_errordetails_error_details_proto = out.File
|
||||
file_pkg_rpc_errordetails_error_details_proto_rawDesc = nil
|
||||
file_pkg_rpc_errordetails_error_details_proto_goTypes = nil
|
||||
file_pkg_rpc_errordetails_error_details_proto_depIdxs = nil
|
||||
}
|
||||
|
|
@ -0,0 +1,111 @@
|
|||
// Code generated by protoc-gen-validate. DO NOT EDIT.
|
||||
// source: pkg/rpc/errordetails/error_details.proto
|
||||
|
||||
package errordetails
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/mail"
|
||||
"net/url"
|
||||
"regexp"
|
||||
"strings"
|
||||
"time"
|
||||
"unicode/utf8"
|
||||
|
||||
"google.golang.org/protobuf/types/known/anypb"
|
||||
)
|
||||
|
||||
// ensure the imports are used
|
||||
var (
|
||||
_ = bytes.MinRead
|
||||
_ = errors.New("")
|
||||
_ = fmt.Print
|
||||
_ = utf8.UTFMax
|
||||
_ = (*regexp.Regexp)(nil)
|
||||
_ = (*strings.Reader)(nil)
|
||||
_ = net.IPv4len
|
||||
_ = time.Duration(0)
|
||||
_ = (*url.URL)(nil)
|
||||
_ = (*mail.Address)(nil)
|
||||
_ = anypb.Any{}
|
||||
)
|
||||
|
||||
// Validate checks the field values on SourceError with the rules defined in
|
||||
// the proto definition for this message. If any rules are violated, an error
|
||||
// is returned.
|
||||
func (m *SourceError) Validate() error {
|
||||
if m == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// no validation rules for Temporary
|
||||
|
||||
if v, ok := any(m.GetMetadata()).(interface{ Validate() error }); ok {
|
||||
if err := v.Validate(); err != nil {
|
||||
return SourceErrorValidationError{
|
||||
field: "Metadata",
|
||||
reason: "embedded message failed validation",
|
||||
cause: err,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// SourceErrorValidationError is the validation error returned by
|
||||
// SourceError.Validate if the designated constraints aren't met.
|
||||
type SourceErrorValidationError struct {
|
||||
field string
|
||||
reason string
|
||||
cause error
|
||||
key bool
|
||||
}
|
||||
|
||||
// Field function returns field value.
|
||||
func (e SourceErrorValidationError) Field() string { return e.field }
|
||||
|
||||
// Reason function returns reason value.
|
||||
func (e SourceErrorValidationError) Reason() string { return e.reason }
|
||||
|
||||
// Cause function returns cause value.
|
||||
func (e SourceErrorValidationError) Cause() error { return e.cause }
|
||||
|
||||
// Key function returns key value.
|
||||
func (e SourceErrorValidationError) Key() bool { return e.key }
|
||||
|
||||
// ErrorName returns error name.
|
||||
func (e SourceErrorValidationError) ErrorName() string { return "SourceErrorValidationError" }
|
||||
|
||||
// Error satisfies the builtin error interface
|
||||
func (e SourceErrorValidationError) Error() string {
|
||||
cause := ""
|
||||
if e.cause != nil {
|
||||
cause = fmt.Sprintf(" | caused by: %v", e.cause)
|
||||
}
|
||||
|
||||
key := ""
|
||||
if e.key {
|
||||
key = "key for "
|
||||
}
|
||||
|
||||
return fmt.Sprintf(
|
||||
"invalid %sSourceError.%s: %s%s",
|
||||
key,
|
||||
e.field,
|
||||
e.reason,
|
||||
cause)
|
||||
}
|
||||
|
||||
var _ error = SourceErrorValidationError{}
|
||||
|
||||
var _ interface {
|
||||
Field() string
|
||||
Reason() string
|
||||
Key() bool
|
||||
Cause() error
|
||||
ErrorName() string
|
||||
} = SourceErrorValidationError{}
|
||||
|
|
@ -0,0 +1,30 @@
|
|||
/*
|
||||
* Copyright 2022 The Dragonfly Authors
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
syntax = "proto3";
|
||||
|
||||
package errordetails;
|
||||
|
||||
import "pkg/rpc/base/base.proto";
|
||||
|
||||
option go_package = "d7y.io/dragonfly/v2/pkg/rpc/errordetails";
|
||||
|
||||
message SourceError {
|
||||
bool temporary = 1;
|
||||
// source response metadata, eg: HTTP Status Code, HTTP Status, HTTP Header
|
||||
base.ExtendAttribute metadata = 2;
|
||||
}
|
||||
|
||||
|
|
@ -50,6 +50,8 @@ var (
|
|||
// Content-Range: <unit> */<size> -> TODO
|
||||
contentRangeRegexp = regexp.MustCompile(`bytes (?P<Start>\d+)-(?P<End>\d+)/(?P<Length>(\d*|\*))`)
|
||||
contentRangeRegexpLengthIndex = contentRangeRegexp.SubexpIndex("Length")
|
||||
|
||||
notTemporaryStatusCode = []int{http.StatusUnauthorized, http.StatusForbidden, http.StatusNotFound, http.StatusProxyAuthRequired}
|
||||
)
|
||||
|
||||
func init() {
|
||||
|
|
@ -208,8 +210,21 @@ func (client *httpSourceClient) GetMetadata(request *source.Request) (*source.Me
|
|||
}
|
||||
return &source.Metadata{
|
||||
Header: hdr,
|
||||
Status: resp.Status,
|
||||
StatusCode: resp.StatusCode,
|
||||
SupportRange: resp.StatusCode == http.StatusPartialContent,
|
||||
TotalContentLength: totalContentLength,
|
||||
Validate: func() error {
|
||||
return source.CheckResponseCode(resp.StatusCode, []int{http.StatusOK, http.StatusPartialContent})
|
||||
},
|
||||
Temporary: func() bool {
|
||||
for _, code := range notTemporaryStatusCode {
|
||||
if code == resp.StatusCode {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return false
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
|
|
@ -245,6 +260,14 @@ func (client *httpSourceClient) Download(request *source.Request) (*source.Respo
|
|||
source.WithValidate(func() error {
|
||||
return source.CheckResponseCode(resp.StatusCode, []int{http.StatusOK, http.StatusPartialContent})
|
||||
}),
|
||||
source.WithTemporary(func() bool {
|
||||
for _, code := range notTemporaryStatusCode {
|
||||
if code == resp.StatusCode {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return false
|
||||
}),
|
||||
source.WithHeader(exportPassThroughHeader(resp.Header)),
|
||||
source.WithExpireInfo(
|
||||
source.ExpireInfo{
|
||||
|
|
|
|||
|
|
@ -17,6 +17,8 @@
|
|||
package source
|
||||
|
||||
type Metadata struct {
|
||||
Status string
|
||||
StatusCode int
|
||||
Header Header
|
||||
// SupportRange indicates source supports partial download, like Range in http request
|
||||
SupportRange bool
|
||||
|
|
@ -27,4 +29,7 @@ type Metadata struct {
|
|||
// Content-Range: bytes 0-9/2443
|
||||
// 2443 is the TotalContentLength, 10 is the ContentLength
|
||||
TotalContentLength int64
|
||||
|
||||
Validate func() error
|
||||
Temporary func() bool
|
||||
}
|
||||
|
|
|
|||
|
|
@ -32,9 +32,11 @@ type Response struct {
|
|||
Header Header
|
||||
Body io.ReadCloser
|
||||
ContentLength int64
|
||||
// validate this response is okay to transfer in p2p network, like status 200 or 206 in http is valid to do this,
|
||||
// Validate this response is okay to transfer in p2p network, like status 200 or 206 in http is valid to do this,
|
||||
// otherwise return status code to original client
|
||||
Validate func() error
|
||||
// Temporary check the error whether the error is temporary, if is true, we can retry it later
|
||||
Temporary func() bool
|
||||
}
|
||||
|
||||
func NewResponse(rc io.ReadCloser, opts ...func(*Response)) *Response {
|
||||
|
|
@ -98,6 +100,12 @@ func WithValidate(validate func() error) func(*Response) {
|
|||
}
|
||||
}
|
||||
|
||||
func WithTemporary(temporary func() bool) func(*Response) {
|
||||
return func(resp *Response) {
|
||||
resp.Temporary = temporary
|
||||
}
|
||||
}
|
||||
|
||||
func (resp *Response) ExpireInfo() ExpireInfo {
|
||||
return ExpireInfo{
|
||||
LastModified: resp.Header.Get(LastModified),
|
||||
|
|
|
|||
Loading…
Reference in New Issue