From 4ea339a33d431aa9218e04cb60cc63f87aa946d1 Mon Sep 17 00:00:00 2001 From: Cathy Date: Mon, 19 Oct 2020 11:10:33 -0400 Subject: [PATCH] colexec, colflow: add network latency stat to streams and refactor This commit adds the information about network latency from an outbox to inbox on EXPLAIN ANALYZE diagrams. This commit also adds the LatencyGetter helper to store node latencies in a map. Rather than updating the latency map multiple times per query, the latency map is updated if it hasn't been updated in over one second. This commit refactors the VectorizedStatsCollector type into a base struct and a specialized stats collector, NetworkVectorizedStatsCollector, for collecting network latency on streams. Since the networkReader is only used for the case where stats are collected for streams, this refactor avoids the need to pass in nils for other instance where stats collectors are used. This commit also adds a separate implementation of the SpanStats interface called VectorizedInboxStats. This change allows for us to choose which stats to display on EXPLAIN ANALYZE diagrams for streams. Release note (sql change): EXPLAIN ANALYZE diagrams now contain "network latency" information on streams. --- pkg/server/server_sql.go | 4 + pkg/server/serverpb/BUILD.bazel | 1 + pkg/server/serverpb/status.go | 49 +++ pkg/sql/colexec/execpb/stats.go | 44 +++ pkg/sql/colexec/execpb/stats.pb.go | 249 +++++++++++-- pkg/sql/colexec/execpb/stats.proto | 8 + pkg/sql/colexec/stats.go | 187 +++++++--- pkg/sql/colexec/stats_test.go | 12 +- pkg/sql/colflow/BUILD.bazel | 1 + pkg/sql/colflow/vectorized_flow.go | 60 ++- pkg/sql/colflow/vectorized_flow_test.go | 11 +- pkg/sql/execinfra/BUILD.bazel | 1 + pkg/sql/execinfra/server_config.go | 3 + pkg/sql/execinfrapb/data.pb.go | 348 ++++++++++-------- pkg/sql/execinfrapb/data.proto | 4 + pkg/sql/execstats/traceanalyzer.go | 6 +- .../testdata/logic_test/dist_vectorize | 4 +- .../testdata/logic_test/explain_analyze_plans | 6 +- pkg/sql/physicalplan/physical_plan.go | 3 +- 19 files changed, 734 insertions(+), 267 deletions(-) diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 38ce3eb40f80..8770917e9702 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -355,6 +355,10 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) { RPCContext: cfg.rpcContext, Stopper: cfg.stopper, + LatencyGetter: &serverpb.LatencyGetter{ + NodesStatusServer: &cfg.nodesStatusServer, + }, + TempStorage: tempEngine, TempStoragePath: cfg.TempStorageConfig.Path, TempFS: tempFS, diff --git a/pkg/server/serverpb/BUILD.bazel b/pkg/server/serverpb/BUILD.bazel index d16f33ebb0aa..7aa3263eb117 100644 --- a/pkg/server/serverpb/BUILD.bazel +++ b/pkg/server/serverpb/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "//pkg/util/errorutil", "//pkg/util/log", "//pkg/util/metric", + "//pkg/util/timeutil", "//pkg/util/uuid", "//vendor/github.com/gogo/protobuf/proto", "//vendor/github.com/gogo/protobuf/sortkeys", diff --git a/pkg/server/serverpb/status.go b/pkg/server/serverpb/status.go index f6183ae6756c..688c6c2d5d66 100644 --- a/pkg/server/serverpb/status.go +++ b/pkg/server/serverpb/status.go @@ -12,8 +12,11 @@ package serverpb import ( context "context" + "time" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/errorutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) // SQLStatusServer is a smaller version of the serverpb.StatusInterface which @@ -61,3 +64,49 @@ func (s *OptionalNodesStatusServer) OptionalNodesStatusServer( } return v.(NodesStatusServer), nil } + +// LatencyGetter stores the map of latencies obtained from the NodesStatusServer. +// These latencies are displayed on the streams of EXPLAIN ANALYZE diagrams. +// This struct is put here to avoid import cycles. +type LatencyGetter struct { + latencyMap map[roachpb.NodeID]map[roachpb.NodeID]int64 + lastUpdatedTime time.Time + NodesStatusServer *OptionalNodesStatusServer +} + +const updateThreshold = 5 * time.Second + +// GetLatency is a helper function that updates the latencies between nodes +// if the time since the last update exceeds the updateThreshold. This function +// returns the latency between the origin node and the target node. +func (lg *LatencyGetter) GetLatency( + ctx context.Context, originNodeID roachpb.NodeID, targetNodeID roachpb.NodeID, +) int64 { + if timeutil.Since(lg.lastUpdatedTime) < updateThreshold { + return lg.latencyMap[originNodeID][targetNodeID] + } + // Update latencies in latencyMap. + ss, err := lg.NodesStatusServer.OptionalNodesStatusServer(errorutil.FeatureNotAvailableToNonSystemTenantsIssue) + if err != nil { + // When latency is 0, it is not shown on EXPLAIN ANALYZE diagrams. + return 0 + } + if lg.latencyMap == nil { + lg.latencyMap = make(map[roachpb.NodeID]map[roachpb.NodeID]int64) + } + response, _ := ss.Nodes(ctx, &NodesRequest{}) + for _, sendingNode := range response.Nodes { + sendingNodeID := sendingNode.Desc.NodeID + if lg.latencyMap[sendingNodeID] == nil { + lg.latencyMap[sendingNodeID] = make(map[roachpb.NodeID]int64) + } + for _, receivingNode := range response.Nodes { + receivingNodeID := receivingNode.Desc.NodeID + if sendingNodeID != receivingNodeID { + lg.latencyMap[sendingNodeID][receivingNodeID] = sendingNode.Activity[receivingNodeID].Latency + } + } + } + lg.lastUpdatedTime = timeutil.Now() + return lg.latencyMap[originNodeID][targetNodeID] +} diff --git a/pkg/sql/colexec/execpb/stats.go b/pkg/sql/colexec/execpb/stats.go index 39dd3f1aa5f7..d17f9b3409f4 100644 --- a/pkg/sql/colexec/execpb/stats.go +++ b/pkg/sql/colexec/execpb/stats.go @@ -21,6 +21,8 @@ import ( var _ tracing.SpanStats = &VectorizedStats{} var _ execinfrapb.DistSQLSpanStats = &VectorizedStats{} +var _ tracing.SpanStats = &VectorizedInboxStats{} +var _ execinfrapb.DistSQLSpanStats = &VectorizedInboxStats{} const ( batchesOutputTagSuffix = "output.batches" @@ -31,6 +33,7 @@ const ( maxVecDiskBytesTagSuffix = "disk.vectorized.max" bytesReadTagSuffix = "bytes.read" rowsReadTagSuffix = "rows.read" + networkLatencyTagSuffix = "network.latency" ) // Stats is part of SpanStats interface. @@ -57,6 +60,26 @@ func (vs *VectorizedStats) Stats() map[string]string { return stats } +// Stats is part of SpanStats interface. +func (vs *VectorizedInboxStats) Stats() map[string]string { + stats := map[string]string{ + batchesOutputTagSuffix: fmt.Sprintf("%d", vs.BaseVectorizedStats.NumBatches), + // TODO(cathymw): Have inbox collect its own deserialization time with a + // timer and display deserialization time instead of ioTime. + ioTimeTagSuffix: fmt.Sprintf("%v", vs.BaseVectorizedStats.Time.Round(time.Microsecond)), + } + if vs.BaseVectorizedStats.BytesRead != 0 { + stats[bytesReadTagSuffix] = humanizeutil.IBytes(vs.BaseVectorizedStats.BytesRead) + } + if vs.BaseVectorizedStats.RowsRead != 0 { + stats[rowsReadTagSuffix] = fmt.Sprintf("%d", vs.BaseVectorizedStats.RowsRead) + } + if vs.NetworkLatency != 0 { + stats[networkLatencyTagSuffix] = fmt.Sprintf("%v", time.Duration(vs.NetworkLatency).Round(time.Microsecond)) + } + return stats +} + const ( batchesOutputQueryPlanSuffix = "batches output" tuplesOutputQueryPlanSuffix = "tuples output" @@ -66,6 +89,7 @@ const ( maxVecDiskBytesQueryPlanSuffix = "max vectorized disk allocated" bytesReadQueryPlanSuffix = "bytes read" rowsReadQueryPlanSuffix = "rows read" + networkLatencyQueryPlanSuffix = "network latency" ) // StatsForQueryPlan is part of DistSQLSpanStats interface. @@ -98,3 +122,23 @@ func (vs *VectorizedStats) StatsForQueryPlan() []string { } return stats } + +// StatsForQueryPlan is part of DistSQLSpanStats interface. +func (vs *VectorizedInboxStats) StatsForQueryPlan() []string { + stats := []string{ + fmt.Sprintf("%s: %d", batchesOutputQueryPlanSuffix, vs.BaseVectorizedStats.NumBatches), + fmt.Sprintf("%s: %v", ioTimeQueryPlanSuffix, vs.BaseVectorizedStats.Time.Round(time.Microsecond)), + } + if vs.BaseVectorizedStats.BytesRead != 0 { + stats = append(stats, + fmt.Sprintf("%s: %s", bytesReadQueryPlanSuffix, humanizeutil.IBytes(vs.BaseVectorizedStats.BytesRead))) + } + if vs.BaseVectorizedStats.RowsRead != 0 { + stats = append(stats, fmt.Sprintf("%s: %d", rowsReadQueryPlanSuffix, vs.BaseVectorizedStats.RowsRead)) + } + if vs.NetworkLatency != 0 { + stats = append(stats, + fmt.Sprintf("%s: %v", networkLatencyQueryPlanSuffix, time.Duration(vs.NetworkLatency).Round(time.Microsecond))) + } + return stats +} diff --git a/pkg/sql/colexec/execpb/stats.pb.go b/pkg/sql/colexec/execpb/stats.pb.go index 367bf00df5b8..6e54a45bb704 100644 --- a/pkg/sql/colexec/execpb/stats.pb.go +++ b/pkg/sql/colexec/execpb/stats.pb.go @@ -51,7 +51,7 @@ func (m *VectorizedStats) Reset() { *m = VectorizedStats{} } func (m *VectorizedStats) String() string { return proto.CompactTextString(m) } func (*VectorizedStats) ProtoMessage() {} func (*VectorizedStats) Descriptor() ([]byte, []int) { - return fileDescriptor_stats_1542874cd30fedd3, []int{0} + return fileDescriptor_stats_21106ba1df89f289, []int{0} } func (m *VectorizedStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -76,8 +76,45 @@ func (m *VectorizedStats) XXX_DiscardUnknown() { var xxx_messageInfo_VectorizedStats proto.InternalMessageInfo +type VectorizedInboxStats struct { + BaseVectorizedStats *VectorizedStats `protobuf:"bytes,1,opt,name=base_vectorized_stats,json=baseVectorizedStats,proto3" json:"base_vectorized_stats,omitempty"` + // network_latency is the latency time in nanoseconds between outbox and inbox. + // It is set only when on_stream is true. + NetworkLatency int64 `protobuf:"varint,2,opt,name=network_latency,json=networkLatency,proto3" json:"network_latency,omitempty"` +} + +func (m *VectorizedInboxStats) Reset() { *m = VectorizedInboxStats{} } +func (m *VectorizedInboxStats) String() string { return proto.CompactTextString(m) } +func (*VectorizedInboxStats) ProtoMessage() {} +func (*VectorizedInboxStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_21106ba1df89f289, []int{1} +} +func (m *VectorizedInboxStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *VectorizedInboxStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *VectorizedInboxStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_VectorizedInboxStats.Merge(dst, src) +} +func (m *VectorizedInboxStats) XXX_Size() int { + return m.Size() +} +func (m *VectorizedInboxStats) XXX_DiscardUnknown() { + xxx_messageInfo_VectorizedInboxStats.DiscardUnknown(m) +} + +var xxx_messageInfo_VectorizedInboxStats proto.InternalMessageInfo + func init() { proto.RegisterType((*VectorizedStats)(nil), "cockroach.sql.execpb.VectorizedStats") + proto.RegisterType((*VectorizedInboxStats)(nil), "cockroach.sql.execpb.VectorizedInboxStats") } func (m *VectorizedStats) Marshal() (dAtA []byte, err error) { size := m.Size() @@ -150,6 +187,39 @@ func (m *VectorizedStats) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *VectorizedInboxStats) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *VectorizedInboxStats) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.BaseVectorizedStats != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintStats(dAtA, i, uint64(m.BaseVectorizedStats.Size())) + n2, err := m.BaseVectorizedStats.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + } + if m.NetworkLatency != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintStats(dAtA, i, uint64(m.NetworkLatency)) + } + return i, nil +} + func encodeVarintStats(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -194,6 +264,22 @@ func (m *VectorizedStats) Size() (n int) { return n } +func (m *VectorizedInboxStats) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BaseVectorizedStats != nil { + l = m.BaseVectorizedStats.Size() + n += 1 + l + sovStats(uint64(l)) + } + if m.NetworkLatency != 0 { + n += 1 + sovStats(uint64(m.NetworkLatency)) + } + return n +} + func sovStats(x uint64) (n int) { for { n++ @@ -440,6 +526,108 @@ func (m *VectorizedStats) Unmarshal(dAtA []byte) error { } return nil } +func (m *VectorizedInboxStats) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: VectorizedInboxStats: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: VectorizedInboxStats: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseVectorizedStats", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStats + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.BaseVectorizedStats == nil { + m.BaseVectorizedStats = &VectorizedStats{} + } + if err := m.BaseVectorizedStats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NetworkLatency", wireType) + } + m.NetworkLatency = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NetworkLatency |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipStats(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStats + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipStats(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 @@ -546,33 +734,38 @@ var ( ) func init() { - proto.RegisterFile("sql/colexec/execpb/stats.proto", fileDescriptor_stats_1542874cd30fedd3) + proto.RegisterFile("sql/colexec/execpb/stats.proto", fileDescriptor_stats_21106ba1df89f289) } -var fileDescriptor_stats_1542874cd30fedd3 = []byte{ - // 379 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x54, 0x91, 0x31, 0x8f, 0xd3, 0x30, - 0x1c, 0xc5, 0xe3, 0xdc, 0x5d, 0x48, 0x7d, 0xc3, 0x81, 0x75, 0x42, 0xe6, 0x10, 0x4e, 0xc4, 0x14, - 0x21, 0x94, 0x48, 0x30, 0x30, 0x13, 0x75, 0x61, 0x40, 0x48, 0x01, 0x31, 0xb0, 0x44, 0x8e, 0x6d, - 0x52, 0xab, 0x71, 0xdc, 0xc6, 0x8e, 0x28, 0x7c, 0x0a, 0x46, 0x3e, 0x52, 0xc7, 0x8e, 0x9d, 0x0a, - 0xa4, 0x5f, 0x04, 0xc5, 0x69, 0x25, 0x6e, 0xb1, 0xfc, 0x7f, 0xef, 0x67, 0xfd, 0xdf, 0x93, 0x21, - 0x31, 0xeb, 0x26, 0x63, 0xba, 0x11, 0x1b, 0xc1, 0xb2, 0xf1, 0x58, 0x55, 0x99, 0xb1, 0xd4, 0x9a, - 0x74, 0xd5, 0x69, 0xab, 0xd1, 0x2d, 0xd3, 0x6c, 0xd9, 0x69, 0xca, 0x16, 0xa9, 0x59, 0x37, 0xe9, - 0x44, 0xdc, 0xdd, 0xd6, 0xba, 0xd6, 0x0e, 0xc8, 0xc6, 0xdb, 0xc4, 0xde, 0x91, 0x5a, 0xeb, 0xba, - 0x11, 0x99, 0x9b, 0xaa, 0xfe, 0x6b, 0xc6, 0xfb, 0x8e, 0x5a, 0xa9, 0xdb, 0xc9, 0x7f, 0xbe, 0xf3, - 0xe1, 0xcd, 0x67, 0xc1, 0xac, 0xee, 0xe4, 0x0f, 0xc1, 0x3f, 0x8e, 0x5b, 0xd0, 0x63, 0xe8, 0x4b, - 0x8e, 0x41, 0x0c, 0x92, 0xab, 0x3c, 0x18, 0x0e, 0x91, 0xff, 0x6e, 0x5e, 0xf8, 0x92, 0xa3, 0x08, - 0x5e, 0xb7, 0xbd, 0x2a, 0x2b, 0x6a, 0xd9, 0x42, 0x18, 0xec, 0xc7, 0x20, 0xb9, 0x28, 0x60, 0xdb, - 0xab, 0x7c, 0x52, 0xd0, 0x33, 0x38, 0x4e, 0xa5, 0xed, 0x57, 0x8d, 0x30, 0xf8, 0xc2, 0xf9, 0xb3, - 0xb6, 0x57, 0x9f, 0x9c, 0x80, 0xde, 0xc0, 0x4b, 0x2b, 0x95, 0xc0, 0x97, 0x31, 0x48, 0xae, 0x5f, - 0x3d, 0x49, 0xa7, 0x68, 0xe9, 0x39, 0x5a, 0x3a, 0x3f, 0x45, 0xcb, 0xc3, 0xed, 0x21, 0xf2, 0x7e, - 0xfd, 0x8e, 0x40, 0xe1, 0x1e, 0xb8, 0x40, 0x1a, 0x5f, 0xc5, 0x20, 0x09, 0x4f, 0x81, 0x3e, 0x14, - 0xbe, 0xd4, 0xe8, 0x05, 0x7c, 0xa4, 0xe8, 0xa6, 0xa4, 0x4d, 0xa3, 0x19, 0xb5, 0x82, 0x97, 0x4a, - 0x28, 0x1c, 0xb8, 0xb5, 0x37, 0x8a, 0x6e, 0xde, 0x9e, 0xf5, 0xf7, 0x42, 0xa1, 0x97, 0x10, 0xdd, - 0x67, 0xb9, 0x34, 0x4b, 0xfc, 0xc0, 0xc1, 0x0f, 0xff, 0x87, 0xe7, 0xd2, 0x2c, 0xc7, 0x26, 0xd5, - 0x77, 0x2b, 0x4c, 0xd9, 0x09, 0xca, 0x71, 0x38, 0x35, 0x71, 0x4a, 0x21, 0x28, 0x47, 0x4f, 0xe1, - 0xac, 0xd3, 0xdf, 0x4e, 0xee, 0xcc, 0xb9, 0xe1, 0x28, 0x8c, 0x66, 0x9e, 0x6c, 0xff, 0x12, 0x6f, - 0x3b, 0x10, 0xb0, 0x1b, 0x08, 0xd8, 0x0f, 0x04, 0xfc, 0x19, 0x08, 0xf8, 0x79, 0x24, 0xde, 0xee, - 0x48, 0xbc, 0xfd, 0x91, 0x78, 0x5f, 0x82, 0xe9, 0xcb, 0xaa, 0xc0, 0x55, 0x7f, 0xfd, 0x2f, 0x00, - 0x00, 0xff, 0xff, 0x3d, 0x2c, 0x83, 0x13, 0xf1, 0x01, 0x00, 0x00, +var fileDescriptor_stats_21106ba1df89f289 = []byte{ + // 449 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x51, 0xcf, 0x6f, 0xd3, 0x30, + 0x14, 0x8e, 0xb3, 0xad, 0xb4, 0xae, 0x44, 0xc1, 0x14, 0x14, 0x86, 0x70, 0xab, 0x49, 0x88, 0x08, + 0xa1, 0x44, 0x1a, 0x07, 0xce, 0x54, 0xbd, 0x4c, 0x02, 0x21, 0x05, 0x84, 0x04, 0x97, 0xc8, 0x71, + 0x4c, 0x17, 0x35, 0xce, 0xeb, 0x62, 0x87, 0x65, 0xfc, 0x15, 0x1c, 0x11, 0x7f, 0x51, 0x8f, 0x3d, + 0xee, 0x34, 0x20, 0xfd, 0x47, 0x50, 0xec, 0x8c, 0x1f, 0x15, 0x17, 0xcb, 0xef, 0xfb, 0x3e, 0xfb, + 0x7d, 0xef, 0x7d, 0x98, 0xaa, 0xb3, 0x3c, 0xe4, 0x90, 0x8b, 0x5a, 0xf0, 0xb0, 0x3d, 0x56, 0x49, + 0xa8, 0x34, 0xd3, 0x2a, 0x58, 0x95, 0xa0, 0x81, 0x8c, 0x39, 0xf0, 0x65, 0x09, 0x8c, 0x9f, 0x06, + 0xea, 0x2c, 0x0f, 0xac, 0xe2, 0x70, 0xbc, 0x80, 0x05, 0x18, 0x41, 0xd8, 0xde, 0xac, 0xf6, 0x90, + 0x2e, 0x00, 0x16, 0xb9, 0x08, 0x4d, 0x95, 0x54, 0x1f, 0xc3, 0xb4, 0x2a, 0x99, 0xce, 0xa0, 0xb0, + 0xfc, 0xd1, 0xc6, 0xc5, 0xa3, 0x77, 0x82, 0x6b, 0x28, 0xb3, 0xcf, 0x22, 0x7d, 0xd3, 0x76, 0x21, + 0xf7, 0xb0, 0x9b, 0xa5, 0x1e, 0x9a, 0x22, 0xff, 0x60, 0xd6, 0x6b, 0xae, 0x26, 0xee, 0xc9, 0x3c, + 0x72, 0xb3, 0x94, 0x4c, 0xf0, 0xb0, 0xa8, 0x64, 0x9c, 0x30, 0xcd, 0x4f, 0x85, 0xf2, 0xdc, 0x29, + 0xf2, 0xf7, 0x22, 0x5c, 0x54, 0x72, 0x66, 0x11, 0xf2, 0x10, 0xb7, 0x55, 0xac, 0xab, 0x55, 0x2e, + 0x94, 0xb7, 0x67, 0xf8, 0x41, 0x51, 0xc9, 0xb7, 0x06, 0x20, 0xcf, 0xf1, 0xbe, 0xce, 0xa4, 0xf0, + 0xf6, 0xa7, 0xc8, 0x1f, 0x1e, 0xdf, 0x0f, 0xac, 0xb5, 0xe0, 0xda, 0x5a, 0x30, 0xef, 0xac, 0xcd, + 0xfa, 0xeb, 0xab, 0x89, 0xf3, 0xf5, 0xfb, 0x04, 0x45, 0xe6, 0x81, 0x31, 0x04, 0xde, 0xc1, 0x14, + 0xf9, 0xfd, 0xce, 0xd0, 0xeb, 0xc8, 0xcd, 0x80, 0x3c, 0xc1, 0xb7, 0x25, 0xab, 0x63, 0x96, 0xe7, + 0xc0, 0x99, 0x16, 0x69, 0x2c, 0x85, 0xf4, 0x7a, 0xa6, 0xed, 0x48, 0xb2, 0xfa, 0xc5, 0x35, 0xfe, + 0x4a, 0x48, 0xf2, 0x14, 0x93, 0x7f, 0xb5, 0x69, 0xa6, 0x96, 0xde, 0x0d, 0x23, 0xbe, 0xf5, 0xb7, + 0x78, 0x9e, 0xa9, 0x65, 0x3b, 0x49, 0x72, 0xa1, 0x85, 0x8a, 0x4b, 0xc1, 0x52, 0xaf, 0x6f, 0x27, + 0x31, 0x48, 0x24, 0x58, 0x4a, 0x1e, 0xe0, 0x41, 0x09, 0xe7, 0x1d, 0x3b, 0x30, 0x6c, 0xbf, 0x05, + 0x5a, 0xf2, 0xe8, 0x1b, 0xc2, 0xe3, 0x3f, 0x2b, 0x3d, 0x29, 0x12, 0xa8, 0xed, 0x5e, 0xdf, 0xe3, + 0xbb, 0x09, 0x53, 0x22, 0xfe, 0xf4, 0x9b, 0x8c, 0x4d, 0xac, 0x66, 0xd5, 0xc3, 0xe3, 0x47, 0xc1, + 0xff, 0x72, 0x0d, 0x76, 0xd2, 0x89, 0xee, 0xb4, 0x7f, 0xec, 0x46, 0xf6, 0x18, 0x8f, 0x0a, 0xa1, + 0xcf, 0xa1, 0x5c, 0xc6, 0x39, 0xd3, 0xa2, 0xe0, 0x17, 0x5d, 0x3c, 0x37, 0x3b, 0xf8, 0xa5, 0x45, + 0x67, 0xfe, 0xfa, 0x27, 0x75, 0xd6, 0x0d, 0x45, 0x9b, 0x86, 0xa2, 0xcb, 0x86, 0xa2, 0x1f, 0x0d, + 0x45, 0x5f, 0xb6, 0xd4, 0xd9, 0x6c, 0xa9, 0x73, 0xb9, 0xa5, 0xce, 0x87, 0x9e, 0xed, 0x9b, 0xf4, + 0x4c, 0x2e, 0xcf, 0x7e, 0x05, 0x00, 0x00, 0xff, 0xff, 0xf5, 0x0d, 0xc5, 0x0f, 0x8e, 0x02, 0x00, + 0x00, } diff --git a/pkg/sql/colexec/execpb/stats.proto b/pkg/sql/colexec/execpb/stats.proto index e104c42bf968..03f1a811b3b3 100644 --- a/pkg/sql/colexec/execpb/stats.proto +++ b/pkg/sql/colexec/execpb/stats.proto @@ -37,3 +37,11 @@ message VectorizedStats { // io is true. int64 rows_read = 9; } + +message VectorizedInboxStats { + VectorizedStats base_vectorized_stats = 1; + + // network_latency is the latency time in nanoseconds between outbox and inbox. + // It is set only when on_stream is true. + int64 network_latency = 2; +} diff --git a/pkg/sql/colexec/stats.go b/pkg/sql/colexec/stats.go index 2c1131f19602..7eb9339c91cc 100644 --- a/pkg/sql/colexec/stats.go +++ b/pkg/sql/colexec/stats.go @@ -13,6 +13,7 @@ package colexec import ( "context" "fmt" + "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execpb" @@ -26,12 +27,21 @@ import ( "github.com/cockroachdb/errors" ) -// VectorizedStatsCollector collects VectorizedStats on Operators. -// -// If two Operators are connected (i.e. one is an input to another), the -// corresponding VectorizedStatsCollectors are also "connected" by sharing a -// StopWatch. -type VectorizedStatsCollector struct { +// VectorizedStatsCollector exists so that the vectorizedStatsCollectorsQueue +// in the colflow.vectorizedFlowCreator can hold both +// VectorizedStatsCollectorBase and NetworkVectorizedStatsCollector types. +type VectorizedStatsCollector interface { + OutputStats(ctx context.Context, flowID string, deterministicStats bool) +} + +// ChildStatsCollector gives access to the stopwatches of a +// VectorizedStatsCollector's childStatsCollectors. +type ChildStatsCollector interface { + getElapsedTime() time.Duration +} + +// VectorizedStatsCollectorBase collects VectorizedStats on Operators. +type VectorizedStatsCollectorBase struct { colexecbase.Operator NonExplainable execpb.VectorizedStats @@ -47,29 +57,30 @@ type VectorizedStatsCollector struct { // childStatsCollectors contains the stats collectors for all of the inputs // to the wrapped operator. - childStatsCollectors []*VectorizedStatsCollector + childStatsCollectors []ChildStatsCollector memMonitors []*mon.BytesMonitor diskMonitors []*mon.BytesMonitor } -var _ colexecbase.Operator = &VectorizedStatsCollector{} +// NetworkVectorizedStatsCollector collects VectorizedInboxStats on Inbox. +type NetworkVectorizedStatsCollector struct { + *VectorizedStatsCollectorBase + execpb.VectorizedInboxStats +} -// NewVectorizedStatsCollector creates a new VectorizedStatsCollector which -// wraps 'op' that corresponds to a component with either ProcessorID or -// StreamID 'id' (with 'idTagKey' distinguishing between the two). 'ioReader' -// is a component (either an operator or a wrapped processor) that performs -// IO reads that is present in the chain of operators rooted at 'op'. -func NewVectorizedStatsCollector( +var _ colexecbase.Operator = &VectorizedStatsCollectorBase{} +var _ colexecbase.Operator = &NetworkVectorizedStatsCollector{} + +// initVectorizedStatsCollectorBase initializes the common fields +// of VectorizedStatsCollectorBase for all VectorizedStatsCollectors. +func initVectorizedStatsCollectorBase( op colexecbase.Operator, ioReader execinfra.IOReader, id int32, idTagKey string, inputWatch *timeutil.StopWatch, - memMonitors []*mon.BytesMonitor, - diskMonitors []*mon.BytesMonitor, - inputStatsCollectors []*VectorizedStatsCollector, -) *VectorizedStatsCollector { +) *VectorizedStatsCollectorBase { if inputWatch == nil { colexecerror.InternalError(errors.AssertionFailedf("input watch for VectorizedStatsCollector is nil")) } @@ -87,20 +98,64 @@ func NewVectorizedStatsCollector( ioTime = false } } - return &VectorizedStatsCollector{ - Operator: op, - VectorizedStats: execpb.VectorizedStats{ID: id, IO: ioTime}, - idTagKey: idTagKey, - ioReader: ioReader, - stopwatch: inputWatch, - memMonitors: memMonitors, - diskMonitors: diskMonitors, - childStatsCollectors: inputStatsCollectors, + + return &VectorizedStatsCollectorBase{ + Operator: op, + VectorizedStats: execpb.VectorizedStats{ID: id, IO: ioTime}, + idTagKey: idTagKey, + ioReader: ioReader, + stopwatch: inputWatch, + } +} + +// NewVectorizedStatsCollectorBase creates a new VectorizedStatsCollectorBase +// which wraps 'op' that corresponds to a component with either ProcessorID or +// StreamID 'id' (with 'idTagKey' distinguishing between the two). 'ioReader' +// is a component (either an operator or a wrapped processor) that performs +// IO reads that is present in the chain of operators rooted at 'op'. +func NewVectorizedStatsCollectorBase( + op colexecbase.Operator, + ioReader execinfra.IOReader, + id int32, + idTagKey string, + inputWatch *timeutil.StopWatch, + memMonitors []*mon.BytesMonitor, + diskMonitors []*mon.BytesMonitor, + inputStatsCollectors []ChildStatsCollector, +) *VectorizedStatsCollectorBase { + vsc := initVectorizedStatsCollectorBase(op, ioReader, id, idTagKey, inputWatch) + + // TODO(cathymw): Refactor to have specialized stats collectors for + // memory/disk stats and IO operators. + vsc.memMonitors = memMonitors + vsc.diskMonitors = diskMonitors + vsc.childStatsCollectors = inputStatsCollectors + return vsc +} + +// NewNetworkVectorizedStatsCollector creates a new VectorizedStatsCollector +// for streams. In addition to the base stats, NewNetworkVectorizedStatsCollector +// collects the network latency for a stream. +func NewNetworkVectorizedStatsCollector( + op colexecbase.Operator, + ioReader execinfra.IOReader, + id int32, + idTagKey string, + inputWatch *timeutil.StopWatch, + latency int64, +) *NetworkVectorizedStatsCollector { + vscBase := initVectorizedStatsCollectorBase(op, ioReader, id, idTagKey, inputWatch) + return &NetworkVectorizedStatsCollector{ + VectorizedStatsCollectorBase: vscBase, + VectorizedInboxStats: execpb.VectorizedInboxStats{ + BaseVectorizedStats: &vscBase.VectorizedStats, + NetworkLatency: latency, + }, } } // Next is part of the Operator interface. -func (vsc *VectorizedStatsCollector) Next(ctx context.Context) coldata.Batch { +func (vsc *VectorizedStatsCollectorBase) Next(ctx context.Context) coldata.Batch { var batch coldata.Batch vsc.stopwatch.Start() batch = vsc.Operator.Next(ctx) @@ -114,13 +169,13 @@ func (vsc *VectorizedStatsCollector) Next(ctx context.Context) coldata.Batch { // finalizeStats records the time measured by the stop watch into the stats as // well as the memory and disk usage. -func (vsc *VectorizedStatsCollector) finalizeStats() { +func (vsc *VectorizedStatsCollectorBase) finalizeStats() { vsc.Time = vsc.stopwatch.Elapsed() // Subtract the time spent in each of the child stats collectors, to produce // the amount of time that the wrapped operator spent doing work itself, not // including time spent waiting on its inputs. for _, statsCollectors := range vsc.childStatsCollectors { - vsc.Time -= statsCollectors.stopwatch.Elapsed() + vsc.Time -= statsCollectors.getElapsedTime() } for _, memMon := range vsc.memMonitors { vsc.MaxAllocatedMem += memMon.MaximumBytes() @@ -140,8 +195,39 @@ func (vsc *VectorizedStatsCollector) finalizeStats() { } } +func (vsc *VectorizedStatsCollectorBase) createSpan( + ctx context.Context, flowID string, +) *tracing.Span { + // We're creating a new span for every component setting the appropriate + // tag so that it is displayed correctly on the flow diagram. + // TODO(yuzefovich): these spans are created and finished right away which + // is not the way they are supposed to be used, so this should be fixed. + _, span := tracing.ChildSpan(ctx, fmt.Sprintf("%T", vsc.Operator)) + span.SetTag(execinfrapb.FlowIDTagKey, flowID) + span.SetTag(vsc.idTagKey, vsc.ID) + return span +} + +// setNonDeterministicStats sets non-deterministic stats collected by vsc to reduce +// non-determinism in tests. +func (vsc *VectorizedStatsCollectorBase) setNonDeterministicStats() { + vsc.Time = 0 + vsc.MaxAllocatedMem = 0 + vsc.MaxAllocatedDisk = 0 + vsc.NumBatches = 0 + // BytesRead is overridden to a useful value for tests. + vsc.BytesRead = 8 * vsc.NumTuples +} + +// setNonDeterministicStats sets non-deterministic stats collected by nvsc to reduce +// non-determinism in tests. +func (nvsc *NetworkVectorizedStatsCollector) setNonDeterministicStats() { + nvsc.VectorizedStatsCollectorBase.setNonDeterministicStats() + nvsc.NetworkLatency = 0 +} + // OutputStats outputs the vectorized stats collected by vsc into ctx. -func (vsc *VectorizedStatsCollector) OutputStats( +func (vsc *VectorizedStatsCollectorBase) OutputStats( ctx context.Context, flowID string, deterministicStats bool, ) { if vsc.ID < 0 { @@ -149,22 +235,35 @@ func (vsc *VectorizedStatsCollector) OutputStats( // component. return } - // We're creating a new span for every component setting the appropriate - // tag so that it is displayed correctly on the flow diagram. - // TODO(yuzefovich): these spans are created and finished right away which - // is not the way they are supposed to be used, so this should be fixed. - _, span := tracing.ChildSpan(ctx, fmt.Sprintf("%T", vsc.Operator)) - span.SetTag(execinfrapb.FlowIDTagKey, flowID) - span.SetTag(vsc.idTagKey, vsc.ID) + span := vsc.createSpan(ctx, flowID) vsc.finalizeStats() if deterministicStats { - vsc.VectorizedStats.Time = 0 - vsc.MaxAllocatedMem = 0 - vsc.MaxAllocatedDisk = 0 - vsc.NumBatches = 0 - // BytesRead is overridden to a useful value for tests. - vsc.BytesRead = 8 * vsc.NumTuples + vsc.setNonDeterministicStats() } span.SetSpanStats(&vsc.VectorizedStats) span.Finish() } + +// OutputStats outputs the vectorized stats collected by nvsc into ctx. +func (nvsc *NetworkVectorizedStatsCollector) OutputStats( + ctx context.Context, flowID string, deterministicStats bool, +) { + if nvsc.ID < 0 { + // Ignore this stats collector since it is not associated with any + // component. + return + } + span := nvsc.createSpan(ctx, flowID) + nvsc.finalizeStats() + if deterministicStats { + nvsc.setNonDeterministicStats() + } + span.SetSpanStats(&nvsc.VectorizedInboxStats) + span.Finish() +} + +// getElapsedTime is a getter that returns the elapsed time in +// childStatsCollectors. +func (vsc *VectorizedStatsCollectorBase) getElapsedTime() time.Duration { + return vsc.stopwatch.Elapsed() +} diff --git a/pkg/sql/colexec/stats_test.go b/pkg/sql/colexec/stats_test.go index 45ee7450285d..b0b556344846 100644 --- a/pkg/sql/colexec/stats_test.go +++ b/pkg/sql/colexec/stats_test.go @@ -33,7 +33,7 @@ func TestNumBatches(t *testing.T) { defer log.Scope(t).Close(t) nBatches := 10 noop := NewNoop(makeFiniteChunksSourceWithBatchSize(nBatches, coldata.BatchSize())) - vsc := NewVectorizedStatsCollector( + vsc := NewVectorizedStatsCollectorBase( noop, nil /* ioReader */, 0 /* id */, execinfrapb.ProcessorIDTagKey, timeutil.NewStopWatch(), nil /* memMonitors */, nil, /* diskMonitors */ nil, /* inputStatsCollectors */ @@ -55,7 +55,7 @@ func TestNumTuples(t *testing.T) { nBatches := 10 for _, batchSize := range []int{1, 16, 1024} { noop := NewNoop(makeFiniteChunksSourceWithBatchSize(nBatches, batchSize)) - vsc := NewVectorizedStatsCollector( + vsc := NewVectorizedStatsCollectorBase( noop, nil /* ioReader */, 0 /* id */, execinfrapb.ProcessorIDTagKey, timeutil.NewStopWatch(), nil /* memMonitors */, nil, /* diskMonitors */ nil, /* inputStatsCollectors */ @@ -87,7 +87,7 @@ func TestVectorizedStatsCollector(t *testing.T) { OneInputNode: NewOneInputNode(makeFiniteChunksSourceWithBatchSize(nBatches, coldata.BatchSize())), timeSource: timeSource, } - leftInput := NewVectorizedStatsCollector( + leftInput := NewVectorizedStatsCollectorBase( leftSource, nil /* ioReader */, 0 /* id */, execinfrapb.ProcessorIDTagKey, timeutil.NewTestStopWatch(timeSource.Now), nil /* memMonitors */, nil, /* diskMonitors */ nil, /* inputStatsCollectors */ @@ -96,7 +96,7 @@ func TestVectorizedStatsCollector(t *testing.T) { OneInputNode: NewOneInputNode(makeFiniteChunksSourceWithBatchSize(nBatches, coldata.BatchSize())), timeSource: timeSource, } - rightInput := NewVectorizedStatsCollector( + rightInput := NewVectorizedStatsCollectorBase( rightSource, nil /* ioReader */, 1 /* id */, execinfrapb.ProcessorIDTagKey, timeutil.NewTestStopWatch(timeSource.Now), nil /* memMonitors */, nil, /* diskMonitors */ nil, /* inputStatsCollectors */ @@ -117,10 +117,10 @@ func TestVectorizedStatsCollector(t *testing.T) { timeSource: timeSource, } - mjStatsCollector := NewVectorizedStatsCollector( + mjStatsCollector := NewVectorizedStatsCollectorBase( timeAdvancingMergeJoiner, nil /* ioReader */, 2 /* id */, execinfrapb.ProcessorIDTagKey, mjInputWatch, nil /* memMonitors */, nil, /* diskMonitors */ - []*VectorizedStatsCollector{leftInput, rightInput}, + []ChildStatsCollector{leftInput, rightInput}, ) // The inputs are identical, so the merge joiner should output diff --git a/pkg/sql/colflow/BUILD.bazel b/pkg/sql/colflow/BUILD.bazel index 5237bff7caf0..ee5db090a29d 100644 --- a/pkg/sql/colflow/BUILD.bazel +++ b/pkg/sql/colflow/BUILD.bazel @@ -62,6 +62,7 @@ go_test( "//pkg/security", "//pkg/security/securitytest", "//pkg/server", + "//pkg/server/serverpb", "//pkg/settings/cluster", "//pkg/sql/catalog/catalogkv", "//pkg/sql/colcontainer", diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index 132d9c98a9e0..1182973e591a 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -340,18 +340,18 @@ func (f *vectorizedFlow) Cleanup(ctx context.Context) { f.Release() } -// wrapWithVectorizedStatsCollector creates a new -// colexec.VectorizedStatsCollector that wraps op and connects the newly +// wrapWithVectorizedStatsCollectorBase creates a new +// colexec.VectorizedStatsCollectorBase that wraps op and connects the newly // created wrapper with those corresponding to operators in inputs (the latter // must have already been wrapped). -func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollector( +func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollectorBase( op colexecbase.Operator, ioReader execinfra.IOReader, inputs []colexecbase.Operator, id int32, idTagKey string, monitors []*mon.BytesMonitor, -) (*colexec.VectorizedStatsCollector, error) { +) (*colexec.VectorizedStatsCollectorBase, error) { inputWatch := timeutil.NewStopWatch() var memMonitors, diskMonitors []*mon.BytesMonitor for _, m := range monitors { @@ -361,15 +361,15 @@ func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollector( memMonitors = append(memMonitors, m) } } - inputStatsCollectors := make([]*colexec.VectorizedStatsCollector, len(inputs)) + inputStatsCollectors := make([]colexec.ChildStatsCollector, len(inputs)) for i, input := range inputs { - sc, ok := input.(*colexec.VectorizedStatsCollector) + sc, ok := input.(colexec.ChildStatsCollector) if !ok { return nil, errors.New("unexpectedly an input is not collecting stats") } inputStatsCollectors[i] = sc } - vsc := colexec.NewVectorizedStatsCollector( + vsc := colexec.NewVectorizedStatsCollectorBase( op, ioReader, id, idTagKey, inputWatch, memMonitors, diskMonitors, inputStatsCollectors, ) @@ -377,13 +377,26 @@ func (s *vectorizedFlowCreator) wrapWithVectorizedStatsCollector( return vsc, nil } +// wrapWithNetworkVectorizedStatsCollector creates a new +// colexec.NetworkVectorizedStatsCollector that wraps op. +func (s *vectorizedFlowCreator) wrapWithNetworkVectorizedStatsCollector( + inbox *colrpc.Inbox, id int32, idTagKey string, latency int64, +) (*colexec.NetworkVectorizedStatsCollector, error) { + inputWatch := timeutil.NewStopWatch() + op := colexecbase.Operator(inbox) + ioReader := execinfra.IOReader(inbox) + nvsc := colexec.NewNetworkVectorizedStatsCollector(op, ioReader, id, idTagKey, inputWatch, latency) + s.vectorizedStatsCollectorsQueue = append(s.vectorizedStatsCollectorsQueue, nvsc) + return nvsc, nil +} + // finishVectorizedStatsCollectors finishes the given stats collectors and // outputs their stats to the trace contained in the ctx's span. func finishVectorizedStatsCollectors( ctx context.Context, flowID execinfrapb.FlowID, deterministicStats bool, - vectorizedStatsCollectors []*colexec.VectorizedStatsCollector, + vectorizedStatsCollectors []colexec.VectorizedStatsCollector, ) { flowIDString := flowID.String() for _, vsc := range vectorizedStatsCollectors { @@ -461,7 +474,7 @@ type vectorizedFlowCreator struct { streamIDToInputOp map[execinfrapb.StreamID]opDAGWithMetaSources recordingStats bool - vectorizedStatsCollectorsQueue []*colexec.VectorizedStatsCollector + vectorizedStatsCollectorsQueue []colexec.VectorizedStatsCollector waitGroup *sync.WaitGroup syncFlowConsumer execinfra.RowReceiver nodeDialer *nodedialer.Dialer @@ -516,7 +529,7 @@ func newVectorizedFlowCreator( remoteComponentCreator: componentCreator, streamIDToInputOp: make(map[execinfrapb.StreamID]opDAGWithMetaSources), recordingStats: recordingStats, - vectorizedStatsCollectorsQueue: make([]*colexec.VectorizedStatsCollector, 0, 2), + vectorizedStatsCollectorsQueue: make([]colexec.VectorizedStatsCollector, 0, 2), waitGroup: waitGroup, syncFlowConsumer: syncFlowConsumer, nodeDialer: nodeDialer, @@ -593,6 +606,7 @@ func (s *vectorizedFlowCreator) setupRemoteOutputStream( if err != nil { return nil, err } + atomic.AddInt32(&s.numOutboxes, 1) run := func(ctx context.Context, cancelFn context.CancelFunc) { // cancelFn is the cancellation function of the context of the whole @@ -696,7 +710,7 @@ func (s *vectorizedFlowCreator) setupRouter( // stats. This is mostly for compatibility but will provide some useful // information (e.g. output stall time). var err error - localOp, err = s.wrapWithVectorizedStatsCollector( + localOp, err = s.wrapWithVectorizedStatsCollectorBase( op, nil /* ioReader */, nil, /* inputs */ int32(stream.StreamID), execinfrapb.StreamIDTagKey, mons, ) @@ -757,18 +771,28 @@ func (s *vectorizedFlowCreator) setupInput( if err := s.checkInboundStreamID(inputStream.StreamID); err != nil { return nil, nil, nil, err } + + var latency int64 + // If LatencyGetter doesn't exist, latency's nil value of 0 is used. + // If latency is 0, it is not included in the displayed stats for + // EXPLAIN ANALYZE diagrams. + if flowCtx.Cfg.LatencyGetter != nil { + latency = flowCtx.Cfg.LatencyGetter.GetLatency(ctx, inputStream.OriginNodeID, inputStream.TargetNodeID) + } + inbox, err := s.remoteComponentCreator.newInbox( ctx, colmem.NewAllocator(ctx, s.newStreamingMemAccount(flowCtx), factory), input.ColumnTypes, inputStream.StreamID, ) + if err != nil { return nil, nil, nil, err } s.addStreamEndpoint(inputStream.StreamID, inbox, s.waitGroup) op := colexecbase.Operator(inbox) if s.recordingStats { - op, err = s.wrapWithVectorizedStatsCollector( - inbox, inbox, nil /* inputs */, int32(inputStream.StreamID), - execinfrapb.StreamIDTagKey, nil, /* monitors */ + op, err = s.wrapWithNetworkVectorizedStatsCollector( + inbox, int32(inputStream.StreamID), + execinfrapb.StreamIDTagKey, latency, ) if err != nil { return nil, nil, nil, err @@ -824,7 +848,7 @@ func (s *vectorizedFlowCreator) setupInput( // TODO(asubiotto): Once we have IDs for synchronizers, plumb them into // this stats collector to display stats. var err error - op, err = s.wrapWithVectorizedStatsCollector( + op, err = s.wrapWithVectorizedStatsCollectorBase( op, nil /* ioReader */, statsInputsAsOps, -1, /* id */ "" /* idTagKey */, nil, /* monitors */ ) @@ -882,7 +906,7 @@ func (s *vectorizedFlowCreator) setupOutput( if s.recordingStats { // If recording stats, we add a metadata source that will generate all // stats data as metadata for the stats collectors created so far. - vscs := append([]*colexec.VectorizedStatsCollector(nil), s.vectorizedStatsCollectorsQueue...) + vscs := append([]colexec.VectorizedStatsCollector(nil), s.vectorizedStatsCollectorsQueue...) s.vectorizedStatsCollectorsQueue = s.vectorizedStatsCollectorsQueue[:0] metadataSourcesQueue = append( metadataSourcesQueue, @@ -914,7 +938,7 @@ func (s *vectorizedFlowCreator) setupOutput( if s.recordingStats { // Make a copy given that vectorizedStatsCollectorsQueue is reset and // appended to. - vscq := append([]*colexec.VectorizedStatsCollector(nil), s.vectorizedStatsCollectorsQueue...) + vscq := append([]colexec.VectorizedStatsCollector(nil), s.vectorizedStatsCollectorsQueue...) outputStatsToTrace = func() { finishVectorizedStatsCollectors( ctx, flowCtx.ID, flowCtx.Cfg.TestingKnobs.DeterministicStats, vscq, @@ -1068,7 +1092,7 @@ func (s *vectorizedFlowCreator) setupFlow( op := result.Op if s.recordingStats { - op, err = s.wrapWithVectorizedStatsCollector( + op, err = s.wrapWithVectorizedStatsCollectorBase( op, result.IOReader, inputs, pspec.ProcessorID, execinfrapb.ProcessorIDTagKey, result.OpMonitors, ) diff --git a/pkg/sql/colflow/vectorized_flow_test.go b/pkg/sql/colflow/vectorized_flow_test.go index a0ba274e01b8..56ae35e0fbd1 100644 --- a/pkg/sql/colflow/vectorized_flow_test.go +++ b/pkg/sql/colflow/vectorized_flow_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" "github.com/cockroachdb/cockroach/pkg/sql/colexec" @@ -215,7 +216,15 @@ func TestDrainOnlyInputDAG(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(st) ctx := context.Background() defer evalCtx.Stop(ctx) - f := &flowinfra.FlowBase{FlowCtx: execinfra.FlowCtx{EvalCtx: &evalCtx, NodeID: base.TestingIDContainer}} + f := &flowinfra.FlowBase{ + FlowCtx: execinfra.FlowCtx{EvalCtx: &evalCtx, + NodeID: base.TestingIDContainer, + Cfg: &execinfra.ServerConfig{ + LatencyGetter: &serverpb.LatencyGetter{ + NodesStatusServer: &serverpb.OptionalNodesStatusServer{}, + }, + }}, + } var wg sync.WaitGroup vfc := newVectorizedFlowCreator( &vectorizedFlowCreatorHelper{f: f}, componentCreator, false, &wg, &execinfra.RowChannel{}, diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index 3c56acf85b88..0484cdb05f79 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -34,6 +34,7 @@ go_library( "//pkg/roachpb", "//pkg/rpc", "//pkg/rpc/nodedialer", + "//pkg/server/serverpb", "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql/catalog/descpb", diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index 9190b1e9f1a7..a5e0fc1df336 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/hydratedtables" @@ -146,6 +147,8 @@ type ServerConfig struct { // HydratedTables is a node-level cache of table descriptors which utilize // user-defined types. HydratedTables *hydratedtables.Cache + + LatencyGetter *serverpb.LatencyGetter } // RuntimeStats is an interface through which the rowexec layer can get diff --git a/pkg/sql/execinfrapb/data.pb.go b/pkg/sql/execinfrapb/data.pb.go index ae9495b421f9..00cb146a179a 100644 --- a/pkg/sql/execinfrapb/data.pb.go +++ b/pkg/sql/execinfrapb/data.pb.go @@ -71,7 +71,7 @@ func (x *Ordering_Column_Direction) UnmarshalJSON(data []byte) error { return nil } func (Ordering_Column_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{2, 0, 0} + return fileDescriptor_data_607bf6adf2d0c389, []int{2, 0, 0} } type StreamEndpointSpec_Type int32 @@ -116,7 +116,7 @@ func (x *StreamEndpointSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (StreamEndpointSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{3, 0} + return fileDescriptor_data_607bf6adf2d0c389, []int{3, 0} } type InputSyncSpec_Type int32 @@ -156,7 +156,7 @@ func (x *InputSyncSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (InputSyncSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{4, 0} + return fileDescriptor_data_607bf6adf2d0c389, []int{4, 0} } type OutputRouterSpec_Type int32 @@ -204,7 +204,7 @@ func (x *OutputRouterSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (OutputRouterSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{5, 0} + return fileDescriptor_data_607bf6adf2d0c389, []int{5, 0} } // Error is a generic representation including a string message. @@ -217,7 +217,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{0} + return fileDescriptor_data_607bf6adf2d0c389, []int{0} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -245,7 +245,7 @@ var xxx_messageInfo_Error proto.InternalMessageInfo func (m *Expression) Reset() { *m = Expression{} } func (*Expression) ProtoMessage() {} func (*Expression) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{1} + return fileDescriptor_data_607bf6adf2d0c389, []int{1} } func (m *Expression) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -280,7 +280,7 @@ func (m *Ordering) Reset() { *m = Ordering{} } func (m *Ordering) String() string { return proto.CompactTextString(m) } func (*Ordering) ProtoMessage() {} func (*Ordering) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{2} + return fileDescriptor_data_607bf6adf2d0c389, []int{2} } func (m *Ordering) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -314,7 +314,7 @@ func (m *Ordering_Column) Reset() { *m = Ordering_Column{} } func (m *Ordering_Column) String() string { return proto.CompactTextString(m) } func (*Ordering_Column) ProtoMessage() {} func (*Ordering_Column) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{2, 0} + return fileDescriptor_data_607bf6adf2d0c389, []int{2, 0} } func (m *Ordering_Column) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -355,13 +355,15 @@ type StreamEndpointSpec struct { StreamID StreamID `protobuf:"varint,2,opt,name=stream_id,json=streamId,casttype=StreamID" json:"stream_id"` // Node ID of the target host, only used for outgoing REMOTE streams. TargetNodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,4,opt,name=target_node_id,json=targetNodeId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"target_node_id"` + // Node ID of the origin node, only used for REMOTE streams. + OriginNodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,5,opt,name=origin_node_id,json=originNodeId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"origin_node_id"` } func (m *StreamEndpointSpec) Reset() { *m = StreamEndpointSpec{} } func (m *StreamEndpointSpec) String() string { return proto.CompactTextString(m) } func (*StreamEndpointSpec) ProtoMessage() {} func (*StreamEndpointSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{3} + return fileDescriptor_data_607bf6adf2d0c389, []int{3} } func (m *StreamEndpointSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -400,7 +402,7 @@ func (m *InputSyncSpec) Reset() { *m = InputSyncSpec{} } func (m *InputSyncSpec) String() string { return proto.CompactTextString(m) } func (*InputSyncSpec) ProtoMessage() {} func (*InputSyncSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{4} + return fileDescriptor_data_607bf6adf2d0c389, []int{4} } func (m *InputSyncSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -444,7 +446,7 @@ func (m *OutputRouterSpec) Reset() { *m = OutputRouterSpec{} } func (m *OutputRouterSpec) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec) ProtoMessage() {} func (*OutputRouterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{5} + return fileDescriptor_data_607bf6adf2d0c389, []int{5} } func (m *OutputRouterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -487,7 +489,7 @@ func (m *OutputRouterSpec_RangeRouterSpec) Reset() { *m = OutputRouterSp func (m *OutputRouterSpec_RangeRouterSpec) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec_RangeRouterSpec) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{5, 0} + return fileDescriptor_data_607bf6adf2d0c389, []int{5, 0} } func (m *OutputRouterSpec_RangeRouterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -529,7 +531,7 @@ func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) String() string { } func (*OutputRouterSpec_RangeRouterSpec_ColumnEncoding) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec_ColumnEncoding) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{5, 0, 0} + return fileDescriptor_data_607bf6adf2d0c389, []int{5, 0, 0} } func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -566,7 +568,7 @@ func (m *OutputRouterSpec_RangeRouterSpec_Span) Reset() { *m = OutputRou func (m *OutputRouterSpec_RangeRouterSpec_Span) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec_RangeRouterSpec_Span) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec_Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{5, 0, 1} + return fileDescriptor_data_607bf6adf2d0c389, []int{5, 0, 1} } func (m *OutputRouterSpec_RangeRouterSpec_Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -600,7 +602,7 @@ func (m *DatumInfo) Reset() { *m = DatumInfo{} } func (m *DatumInfo) String() string { return proto.CompactTextString(m) } func (*DatumInfo) ProtoMessage() {} func (*DatumInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{6} + return fileDescriptor_data_607bf6adf2d0c389, []int{6} } func (m *DatumInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -635,7 +637,7 @@ func (m *ProducerHeader) Reset() { *m = ProducerHeader{} } func (m *ProducerHeader) String() string { return proto.CompactTextString(m) } func (*ProducerHeader) ProtoMessage() {} func (*ProducerHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{7} + return fileDescriptor_data_607bf6adf2d0c389, []int{7} } func (m *ProducerHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -678,7 +680,7 @@ func (m *ProducerData) Reset() { *m = ProducerData{} } func (m *ProducerData) String() string { return proto.CompactTextString(m) } func (*ProducerData) ProtoMessage() {} func (*ProducerData) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{8} + return fileDescriptor_data_607bf6adf2d0c389, []int{8} } func (m *ProducerData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +722,7 @@ func (m *ProducerMessage) Reset() { *m = ProducerMessage{} } func (m *ProducerMessage) String() string { return proto.CompactTextString(m) } func (*ProducerMessage) ProtoMessage() {} func (*ProducerMessage) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{9} + return fileDescriptor_data_607bf6adf2d0c389, []int{9} } func (m *ProducerMessage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -766,7 +768,7 @@ func (m *RemoteProducerMetadata) Reset() { *m = RemoteProducerMetadata{} func (m *RemoteProducerMetadata) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata) ProtoMessage() {} func (*RemoteProducerMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{10} + return fileDescriptor_data_607bf6adf2d0c389, []int{10} } func (m *RemoteProducerMetadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1090,7 +1092,7 @@ func (m *RemoteProducerMetadata_RangeInfos) Reset() { *m = RemoteProduce func (m *RemoteProducerMetadata_RangeInfos) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_RangeInfos) ProtoMessage() {} func (*RemoteProducerMetadata_RangeInfos) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{10, 0} + return fileDescriptor_data_607bf6adf2d0c389, []int{10, 0} } func (m *RemoteProducerMetadata_RangeInfos) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1123,7 +1125,7 @@ func (m *RemoteProducerMetadata_TraceData) Reset() { *m = RemoteProducer func (m *RemoteProducerMetadata_TraceData) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_TraceData) ProtoMessage() {} func (*RemoteProducerMetadata_TraceData) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{10, 1} + return fileDescriptor_data_607bf6adf2d0c389, []int{10, 1} } func (m *RemoteProducerMetadata_TraceData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1164,7 +1166,7 @@ func (m *RemoteProducerMetadata_RowNum) Reset() { *m = RemoteProducerMet func (m *RemoteProducerMetadata_RowNum) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_RowNum) ProtoMessage() {} func (*RemoteProducerMetadata_RowNum) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{10, 2} + return fileDescriptor_data_607bf6adf2d0c389, []int{10, 2} } func (m *RemoteProducerMetadata_RowNum) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1204,7 +1206,7 @@ func (m *RemoteProducerMetadata_SamplerProgress) Reset() { func (m *RemoteProducerMetadata_SamplerProgress) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_SamplerProgress) ProtoMessage() {} func (*RemoteProducerMetadata_SamplerProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{10, 3} + return fileDescriptor_data_607bf6adf2d0c389, []int{10, 3} } func (m *RemoteProducerMetadata_SamplerProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1245,7 +1247,7 @@ func (m *RemoteProducerMetadata_BulkProcessorProgress) String() string { } func (*RemoteProducerMetadata_BulkProcessorProgress) ProtoMessage() {} func (*RemoteProducerMetadata_BulkProcessorProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{10, 4} + return fileDescriptor_data_607bf6adf2d0c389, []int{10, 4} } func (m *RemoteProducerMetadata_BulkProcessorProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1282,7 +1284,7 @@ func (m *RemoteProducerMetadata_Metrics) Reset() { *m = RemoteProducerMe func (m *RemoteProducerMetadata_Metrics) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_Metrics) ProtoMessage() {} func (*RemoteProducerMetadata_Metrics) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{10, 5} + return fileDescriptor_data_607bf6adf2d0c389, []int{10, 5} } func (m *RemoteProducerMetadata_Metrics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1322,7 +1324,7 @@ func (m *DistSQLVersionGossipInfo) Reset() { *m = DistSQLVersionGossipIn func (m *DistSQLVersionGossipInfo) String() string { return proto.CompactTextString(m) } func (*DistSQLVersionGossipInfo) ProtoMessage() {} func (*DistSQLVersionGossipInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{11} + return fileDescriptor_data_607bf6adf2d0c389, []int{11} } func (m *DistSQLVersionGossipInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1358,7 +1360,7 @@ func (m *DistSQLDrainingInfo) Reset() { *m = DistSQLDrainingInfo{} } func (m *DistSQLDrainingInfo) String() string { return proto.CompactTextString(m) } func (*DistSQLDrainingInfo) ProtoMessage() {} func (*DistSQLDrainingInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_a11511dae3716e94, []int{12} + return fileDescriptor_data_607bf6adf2d0c389, []int{12} } func (m *DistSQLDrainingInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1602,6 +1604,9 @@ func (m *StreamEndpointSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x20 i++ i = encodeVarintData(dAtA, i, uint64(m.TargetNodeID)) + dAtA[i] = 0x28 + i++ + i = encodeVarintData(dAtA, i, uint64(m.OriginNodeID)) return i, nil } @@ -2447,6 +2452,7 @@ func (m *StreamEndpointSpec) Size() (n int) { n += 1 + sovData(uint64(m.Type)) n += 1 + sovData(uint64(m.StreamID)) n += 1 + sovData(uint64(m.TargetNodeID)) + n += 1 + sovData(uint64(m.OriginNodeID)) return n } @@ -3304,6 +3310,25 @@ func (m *StreamEndpointSpec) Unmarshal(dAtA []byte) error { break } } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field OriginNodeID", wireType) + } + m.OriginNodeID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.OriginNodeID |= (github_com_cockroachdb_cockroach_pkg_roachpb.NodeID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipData(dAtA[iNdEx:]) @@ -5858,138 +5883,139 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/execinfrapb/data.proto", fileDescriptor_data_a11511dae3716e94) } - -var fileDescriptor_data_a11511dae3716e94 = []byte{ - // 2065 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0xcb, 0x73, 0x1b, 0xc7, - 0xd1, 0xc7, 0x82, 0x20, 0x1e, 0xcd, 0x17, 0x38, 0x9f, 0x2c, 0xe3, 0x43, 0x14, 0x92, 0x86, 0x52, - 0x2e, 0xc5, 0xb1, 0x17, 0x12, 0x75, 0x90, 0xc3, 0x24, 0x25, 0x11, 0x04, 0x28, 0x40, 0xa6, 0x48, - 0x7a, 0x41, 0xc5, 0x25, 0x3b, 0xae, 0xad, 0xc5, 0xee, 0x00, 0x5c, 0x6b, 0x5f, 0x9a, 0x99, 0x15, - 0xc9, 0x4b, 0x2a, 0x55, 0xc9, 0x31, 0x87, 0x9c, 0x52, 0x39, 0x3a, 0x97, 0x5c, 0x72, 0xc8, 0xbf, - 0x90, 0x53, 0xa2, 0x53, 0xca, 0x47, 0x55, 0x0e, 0xaa, 0x84, 0xba, 0xe4, 0x98, 0x6b, 0x7c, 0x4a, - 0xcd, 0x0b, 0x2f, 0x91, 0x52, 0x68, 0xf9, 0x02, 0xcc, 0xf4, 0xf4, 0xef, 0x37, 0xdd, 0x3d, 0x3d, - 0x3d, 0x33, 0x0b, 0x55, 0xfa, 0x38, 0xa8, 0xe3, 0x63, 0xec, 0xfa, 0x51, 0x9f, 0x38, 0x49, 0xaf, - 0xee, 0x39, 0xcc, 0x31, 0x13, 0x12, 0xb3, 0x18, 0x55, 0xdc, 0xd8, 0x7d, 0x44, 0x62, 0xc7, 0x3d, - 0x34, 0xe9, 0xe3, 0xc0, 0xf4, 0x7c, 0xca, 0xe8, 0xe3, 0x80, 0xa4, 0x51, 0xf5, 0xf2, 0x17, 0x71, - 0x8f, 0xd6, 0xf9, 0x4f, 0xd2, 0x13, 0x7f, 0x12, 0x51, 0x5d, 0x16, 0xda, 0x49, 0xaf, 0xee, 0x24, - 0xbe, 0x12, 0x21, 0x2d, 0x1a, 0x11, 0x57, 0x2f, 0x69, 0x19, 0x26, 0x24, 0x26, 0x1a, 0xfc, 0x96, - 0xec, 0x4d, 0x8b, 0x57, 0xb9, 0x85, 0xc9, 0xe0, 0xc8, 0x27, 0xb8, 0x9e, 0x0c, 0xc4, 0xd0, 0xa4, - 0xc2, 0x55, 0xae, 0xe0, 0x3a, 0xcc, 0x09, 0xe2, 0x41, 0xdd, 0xc3, 0xd4, 0x4d, 0x7a, 0x75, 0xca, - 0x48, 0xea, 0xb2, 0x94, 0x60, 0x4f, 0x29, 0xbd, 0x7b, 0x86, 0x12, 0x8e, 0xdc, 0xd8, 0xc3, 0x9e, - 0xed, 0x39, 0x2c, 0x0d, 0xb5, 0x11, 0x5c, 0x8f, 0x9d, 0x24, 0x98, 0xca, 0x5f, 0x25, 0x7e, 0x2f, - 0x65, 0x7e, 0x50, 0x67, 0xc4, 0x71, 0xfd, 0x68, 0xa0, 0xff, 0x93, 0x5e, 0x9d, 0x60, 0x37, 0x26, - 0x9c, 0x83, 0x26, 0x4e, 0xa4, 0xbd, 0x1b, 0xc4, 0x83, 0x58, 0x34, 0xeb, 0xbc, 0xa5, 0xa4, 0x15, - 0xc1, 0x70, 0x18, 0xb8, 0x75, 0xe6, 0x87, 0x98, 0x32, 0x27, 0x4c, 0xd4, 0xc8, 0xff, 0x0f, 0xe2, - 0x78, 0x10, 0xe0, 0xba, 0xe8, 0xf5, 0xd2, 0x7e, 0xdd, 0x89, 0x4e, 0xe4, 0x50, 0xed, 0x53, 0x98, - 0x6d, 0x71, 0x57, 0xd1, 0x6d, 0x80, 0x7e, 0x1a, 0x04, 0xb6, 0x70, 0xbc, 0x32, 0xb3, 0x66, 0x5c, - 0x9b, 0x5b, 0x5f, 0x33, 0x47, 0xeb, 0xa3, 0x43, 0x67, 0xb6, 0xa4, 0x4f, 0x02, 0x65, 0x95, 0x38, - 0x46, 0x34, 0x37, 0x72, 0xbf, 0xfb, 0x72, 0x35, 0x73, 0x2f, 0x57, 0x34, 0xca, 0xd9, 0x7b, 0xb9, - 0x62, 0xb6, 0x3c, 0x53, 0xdb, 0x07, 0x68, 0x1d, 0x27, 0x04, 0x53, 0xea, 0xc7, 0x11, 0x5a, 0x81, - 0xc2, 0x13, 0x4c, 0x78, 0xb3, 0x62, 0xac, 0x19, 0xd7, 0x4a, 0x8d, 0xdc, 0xd3, 0xe7, 0xab, 0x19, - 0x4b, 0x0b, 0x51, 0x05, 0x72, 0xf8, 0x38, 0x21, 0x95, 0xec, 0xd8, 0xa0, 0x90, 0x6c, 0x14, 0x39, - 0xf3, 0xbf, 0x7f, 0xbf, 0x9a, 0xa9, 0xfd, 0x2a, 0x0b, 0xc5, 0x3d, 0xe2, 0x61, 0xe2, 0x47, 0x03, - 0xd4, 0x81, 0x82, 0x1b, 0x07, 0x69, 0x18, 0xd1, 0x8a, 0xb1, 0x36, 0x73, 0x6d, 0x6e, 0xfd, 0xfb, - 0xe6, 0x79, 0xe9, 0x64, 0x6a, 0x90, 0xb9, 0x25, 0x10, 0x7a, 0x6e, 0x85, 0xaf, 0xfe, 0xc1, 0x80, - 0xbc, 0x1c, 0x41, 0xdf, 0x15, 0xac, 0xb6, 0xef, 0x1d, 0x0b, 0x33, 0x17, 0x94, 0x6a, 0xde, 0x8d, - 0x83, 0x8e, 0x77, 0x8c, 0x3e, 0x81, 0x92, 0xe7, 0x13, 0xec, 0x32, 0xee, 0x07, 0x37, 0x75, 0x71, - 0xfd, 0xe6, 0xff, 0x3c, 0xad, 0xd9, 0xd4, 0x50, 0xc5, 0x3a, 0xe2, 0xaa, 0xad, 0x40, 0x69, 0x38, - 0x8a, 0x0a, 0x30, 0xb3, 0xd9, 0xdd, 0x2a, 0x67, 0x50, 0x11, 0x72, 0xcd, 0x56, 0x77, 0xab, 0x6c, - 0x6c, 0xe4, 0xfe, 0xf5, 0xe5, 0xaa, 0xfa, 0xad, 0xfd, 0x35, 0x0b, 0xa8, 0xcb, 0x08, 0x76, 0xc2, - 0x56, 0xe4, 0x25, 0xb1, 0x1f, 0xb1, 0x6e, 0x82, 0x5d, 0xf4, 0x11, 0xe4, 0x78, 0x46, 0x09, 0xbb, - 0x17, 0xd7, 0x6f, 0x9c, 0x6f, 0xd6, 0xcb, 0x58, 0xf3, 0xe0, 0x24, 0xc1, 0x3a, 0xe8, 0x9c, 0x04, - 0xfd, 0x10, 0x4a, 0x54, 0xa8, 0xd9, 0xbe, 0x27, 0x1c, 0x9d, 0x6d, 0x5c, 0xe1, 0xc3, 0xa7, 0xcf, - 0x57, 0x8b, 0x12, 0xdf, 0x69, 0x7e, 0x3d, 0xd6, 0xb6, 0x8a, 0x52, 0xbd, 0xe3, 0xa1, 0xc7, 0xb0, - 0xc8, 0x1c, 0x32, 0xc0, 0xcc, 0x8e, 0x62, 0x0f, 0x73, 0x7c, 0x4e, 0xe0, 0x3f, 0x52, 0xf8, 0xf9, - 0x03, 0x31, 0xba, 0x1b, 0x7b, 0x58, 0x70, 0xdc, 0x1c, 0xf8, 0xec, 0x30, 0xed, 0x99, 0x6e, 0x1c, - 0xd6, 0x87, 0x36, 0x7b, 0xbd, 0x51, 0xbb, 0x9e, 0x3c, 0x1a, 0xd4, 0xd5, 0x8e, 0x36, 0x25, 0xcc, - 0x9a, 0x67, 0x23, 0x12, 0xaf, 0x76, 0x1d, 0x72, 0xdc, 0x03, 0x54, 0x82, 0xd9, 0x9d, 0xbd, 0xad, - 0xcd, 0x9d, 0x72, 0x06, 0x01, 0xe4, 0xad, 0xd6, 0xfd, 0xbd, 0x83, 0x56, 0xd9, 0x40, 0xcb, 0xb0, - 0xd0, 0x7d, 0xb8, 0xbb, 0x65, 0x5b, 0xad, 0xee, 0xfe, 0xde, 0x6e, 0xb7, 0x25, 0x52, 0x74, 0xa6, - 0x9c, 0xab, 0xfd, 0x2d, 0x0b, 0x0b, 0x9d, 0x28, 0x49, 0x59, 0xf7, 0x24, 0x72, 0x45, 0x10, 0xb7, - 0x27, 0x82, 0xf8, 0xfe, 0xf9, 0x41, 0x9c, 0x80, 0xbd, 0x1c, 0xbf, 0x26, 0x14, 0x63, 0xb5, 0xfa, - 0x22, 0x7c, 0x73, 0xeb, 0xb5, 0xd7, 0xe7, 0x89, 0x62, 0x18, 0x22, 0xd1, 0x0e, 0x14, 0x64, 0x58, - 0x69, 0x65, 0x46, 0xe4, 0xf8, 0xfb, 0x17, 0x59, 0x55, 0x9d, 0xe6, 0x8a, 0x02, 0xfd, 0x04, 0xe6, - 0x65, 0xc6, 0xdb, 0xa2, 0xf2, 0x54, 0x72, 0x82, 0xb2, 0x3a, 0x45, 0x49, 0x71, 0x68, 0xca, 0xca, - 0x74, 0x60, 0xcd, 0x49, 0x7d, 0xee, 0x1d, 0xad, 0xd5, 0x54, 0x90, 0x17, 0xa0, 0xf4, 0x60, 0x77, - 0xcf, 0x6a, 0xb6, 0xac, 0x56, 0xb3, 0x9c, 0x41, 0x73, 0x50, 0xd0, 0x1d, 0xa3, 0xf6, 0x9f, 0x3c, - 0x94, 0xf7, 0x52, 0x96, 0xa4, 0xcc, 0x8a, 0x53, 0x86, 0x89, 0x88, 0x69, 0x67, 0x22, 0xa6, 0xf5, - 0x57, 0xc4, 0x61, 0x0a, 0xf9, 0x72, 0x58, 0xc7, 0x02, 0x92, 0x7d, 0xf3, 0x80, 0xbc, 0x03, 0xf3, - 0x87, 0x0e, 0x3d, 0xb4, 0x75, 0x1d, 0xe1, 0x31, 0x5e, 0xb0, 0xe6, 0xb8, 0x4c, 0xee, 0x58, 0x8a, - 0x02, 0x58, 0x26, 0x4e, 0x34, 0xc0, 0x36, 0x11, 0x56, 0xd9, 0x34, 0xc1, 0xae, 0xc8, 0xe7, 0xb9, - 0xf5, 0x8d, 0x0b, 0x38, 0x62, 0x71, 0x8e, 0x51, 0x5f, 0x19, 0xb2, 0x44, 0x26, 0xc5, 0xe8, 0x06, - 0x2c, 0x7b, 0x3e, 0x75, 0x7a, 0x01, 0xb6, 0x7b, 0x69, 0xbf, 0x2f, 0xd3, 0x67, 0x76, 0xcd, 0xb8, - 0x56, 0x54, 0x88, 0xb2, 0x1a, 0x6e, 0xe8, 0xd1, 0xea, 0x5f, 0x66, 0x60, 0x69, 0x8a, 0x1d, 0x7d, - 0x06, 0xb3, 0xfc, 0xb8, 0xd0, 0x85, 0xf1, 0xf6, 0x37, 0x37, 0xd4, 0xec, 0x26, 0x8e, 0xae, 0x56, - 0x92, 0x93, 0x07, 0xcd, 0xc3, 0x7d, 0x27, 0x0d, 0x98, 0xed, 0x61, 0xca, 0x64, 0x71, 0xb0, 0xe6, - 0x94, 0xac, 0x89, 0x29, 0x43, 0x21, 0x94, 0xc4, 0xd1, 0xe7, 0x47, 0x03, 0x9d, 0xb8, 0x9d, 0x37, - 0xb0, 0x41, 0xae, 0x45, 0x4b, 0x31, 0xea, 0xda, 0x39, 0x9c, 0xa1, 0xfa, 0x04, 0x16, 0x27, 0x55, - 0xd0, 0x15, 0xc8, 0xcb, 0x35, 0x7d, 0xa9, 0x88, 0xf3, 0x1a, 0xbf, 0x0d, 0x45, 0x0d, 0x56, 0x35, - 0xfc, 0x7b, 0xd3, 0x7b, 0xe0, 0x71, 0xd0, 0x73, 0x28, 0x36, 0x9b, 0xfc, 0xe0, 0x9e, 0x9a, 0x78, - 0x88, 0xad, 0xee, 0x40, 0x8e, 0x87, 0x07, 0x5d, 0x82, 0x59, 0xca, 0x1c, 0xc2, 0xc4, 0x64, 0xf3, - 0x96, 0xec, 0xa0, 0x32, 0xcc, 0xe0, 0x48, 0xd6, 0xce, 0x79, 0x8b, 0x37, 0xb9, 0x55, 0x32, 0xf3, - 0xc4, 0xf9, 0x3a, 0xab, 0xad, 0x92, 0xb2, 0xda, 0x6d, 0xb5, 0xbd, 0xca, 0x30, 0xbf, 0xbf, 0xd9, - 0xed, 0xda, 0x07, 0x6d, 0x6b, 0xef, 0xc1, 0xdd, 0xb6, 0x2c, 0x65, 0xf7, 0x3b, 0x96, 0xb5, 0x67, - 0x95, 0x0d, 0xbe, 0xdb, 0x1a, 0x0f, 0xed, 0xf6, 0x66, 0xb7, 0x5d, 0xce, 0xa2, 0x79, 0x28, 0x36, - 0x1e, 0xda, 0xd6, 0xe6, 0xee, 0xdd, 0x56, 0x79, 0xa6, 0xf6, 0x4b, 0x03, 0x4a, 0xc2, 0xe0, 0x4e, - 0xd4, 0x8f, 0x27, 0x9c, 0x34, 0xbe, 0xb9, 0x93, 0xc8, 0x54, 0x9b, 0x57, 0x16, 0xb1, 0x57, 0x15, - 0x0b, 0xa1, 0x57, 0xfb, 0x39, 0x2c, 0xee, 0x93, 0xd8, 0x4b, 0x5d, 0x4c, 0xda, 0xd8, 0xf1, 0x30, - 0x41, 0x37, 0xa0, 0xd0, 0x0f, 0xe2, 0x23, 0x7e, 0x10, 0x88, 0x00, 0x35, 0x2a, 0x7c, 0x8a, 0xbf, - 0x3f, 0x5f, 0xcd, 0x6f, 0x07, 0xf1, 0x51, 0xa7, 0x79, 0x3a, 0x6c, 0x59, 0x79, 0xae, 0xd8, 0xf1, - 0xde, 0xe0, 0xf4, 0xa9, 0xfd, 0xd1, 0x80, 0x79, 0x6d, 0x40, 0xd3, 0x61, 0x0e, 0xfa, 0x0e, 0x94, - 0x88, 0x73, 0x64, 0xf7, 0x4e, 0x18, 0xa6, 0x6a, 0x85, 0x8a, 0xc4, 0x39, 0x6a, 0xf0, 0x3e, 0xb2, - 0xa0, 0x18, 0x62, 0xe6, 0xf0, 0xab, 0xa3, 0x2a, 0x28, 0xd7, 0xcf, 0x4f, 0x54, 0x0b, 0x87, 0x31, - 0xc3, 0x9a, 0xfc, 0xbe, 0xc2, 0xe9, 0x88, 0x69, 0x1e, 0xf4, 0x1e, 0x2c, 0x46, 0x69, 0x68, 0xe3, - 0x30, 0x61, 0x27, 0x36, 0x89, 0x8f, 0xe8, 0xc4, 0x72, 0xcf, 0x47, 0x69, 0xd8, 0xe2, 0x43, 0x56, - 0x7c, 0x44, 0x6b, 0xcf, 0x0c, 0x58, 0x1a, 0x11, 0x52, 0xea, 0x0c, 0x30, 0xba, 0x03, 0xf9, 0x43, - 0x11, 0x39, 0x61, 0xed, 0xdc, 0xfa, 0xb5, 0xf3, 0x2d, 0x9a, 0x8c, 0xb4, 0xa5, 0x70, 0x68, 0x13, - 0xf2, 0xec, 0x24, 0x91, 0xe9, 0xcd, 0x7d, 0xba, 0x7a, 0x3e, 0xc3, 0x30, 0x61, 0x74, 0x36, 0x4a, - 0x20, 0xba, 0x03, 0x39, 0x11, 0x14, 0x79, 0x13, 0x7c, 0xf7, 0xf5, 0x26, 0x34, 0x47, 0xa1, 0x10, - 0xc8, 0xda, 0x9f, 0x17, 0xe1, 0xf2, 0xd9, 0x11, 0x43, 0x3f, 0x03, 0x90, 0x45, 0xd5, 0x8f, 0xfa, - 0xb1, 0xf2, 0xf2, 0x47, 0x17, 0x8d, 0xbb, 0x2c, 0x13, 0xdc, 0x74, 0xda, 0xce, 0x58, 0x25, 0xa2, - 0x7b, 0xe8, 0x16, 0xcc, 0xca, 0x5b, 0xac, 0x4c, 0xd9, 0xd5, 0xf3, 0x89, 0xc5, 0xcd, 0xb5, 0x9d, - 0xb1, 0xa4, 0x3e, 0xfa, 0x0c, 0x80, 0x5f, 0xbc, 0xb1, 0x3d, 0xe6, 0xf9, 0xc6, 0x85, 0xcd, 0x3a, - 0xe0, 0x14, 0x3c, 0x1a, 0xdc, 0x2a, 0xa6, 0x3b, 0xe8, 0x13, 0xb8, 0x14, 0x60, 0xa7, 0x6f, 0xb3, - 0xe3, 0xc8, 0xee, 0xfb, 0x91, 0x13, 0xd8, 0x94, 0x39, 0x0c, 0xab, 0xb3, 0x64, 0x7c, 0x6f, 0xea, - 0x9b, 0xce, 0x0e, 0x76, 0xfa, 0x07, 0xc7, 0xd1, 0x36, 0x57, 0xee, 0x72, 0xdd, 0x76, 0xc6, 0x5a, - 0x0e, 0xa6, 0x85, 0xc8, 0x82, 0x02, 0x89, 0x8f, 0xec, 0x28, 0x0d, 0xc5, 0x49, 0x31, 0xb7, 0x7e, - 0xeb, 0xe2, 0x91, 0x8c, 0x8f, 0x76, 0xd3, 0xb0, 0x9d, 0xb1, 0xf2, 0x44, 0xb4, 0x50, 0x08, 0x65, - 0xea, 0x84, 0x49, 0x80, 0x89, 0x9d, 0x90, 0x78, 0xc0, 0xef, 0xf0, 0x95, 0x82, 0x20, 0xbf, 0x73, - 0x61, 0xf2, 0xae, 0x24, 0xda, 0x57, 0x3c, 0xed, 0x8c, 0xb5, 0x44, 0x27, 0x45, 0xe8, 0x00, 0x0a, - 0x21, 0x66, 0xc4, 0x77, 0x69, 0xa5, 0x28, 0x66, 0xf9, 0xf0, 0xc2, 0xb3, 0xdc, 0x97, 0xf8, 0x76, - 0xc6, 0xd2, 0x54, 0xe8, 0x17, 0x06, 0xbc, 0xdd, 0x4b, 0x83, 0x47, 0xdc, 0x05, 0x17, 0x53, 0x1a, - 0x8f, 0x39, 0x53, 0x12, 0xd3, 0x6c, 0x5f, 0x78, 0x9a, 0x46, 0x1a, 0x3c, 0xda, 0xd7, 0x74, 0x63, - 0x2e, 0xbd, 0xd5, 0x3b, 0x6b, 0xa0, 0xba, 0x07, 0x30, 0xca, 0x52, 0xb4, 0x39, 0x95, 0xf6, 0x7c, - 0x6b, 0x5e, 0x39, 0x63, 0xe1, 0x87, 0x10, 0x7d, 0xd4, 0x0d, 0x73, 0xbb, 0xfa, 0x05, 0x94, 0x86, - 0xf9, 0x85, 0x3e, 0x87, 0x25, 0x37, 0x0e, 0x02, 0xec, 0x32, 0xf5, 0x3e, 0xd4, 0x07, 0xbe, 0x39, - 0x46, 0xca, 0x5f, 0x85, 0xa6, 0x7a, 0x4f, 0x9a, 0xc3, 0x77, 0xa5, 0x69, 0xa9, 0x77, 0xe5, 0xd8, - 0xf9, 0xbe, 0x38, 0x24, 0xe3, 0x42, 0x5a, 0x3d, 0x82, 0xbc, 0x4c, 0x0c, 0xf4, 0x01, 0x94, 0x28, - 0x8e, 0x3c, 0x4c, 0x74, 0x0d, 0x2f, 0x35, 0xca, 0xc3, 0x72, 0x2c, 0x06, 0x44, 0x09, 0x96, 0x2d, - 0x8f, 0xbf, 0xa1, 0x74, 0x46, 0x66, 0xc7, 0x0f, 0x3a, 0x95, 0x5c, 0xab, 0x50, 0x0c, 0x1c, 0xca, - 0xec, 0x90, 0x0e, 0xc4, 0x26, 0xd3, 0x77, 0x9b, 0x02, 0x97, 0xde, 0xa7, 0x83, 0x2a, 0x85, 0xa5, - 0xa9, 0xa4, 0x41, 0x3f, 0x80, 0x45, 0x5e, 0x49, 0xf5, 0x52, 0x62, 0x69, 0x46, 0x4e, 0x21, 0x17, - 0xf8, 0xd8, 0xbe, 0x1e, 0x42, 0x37, 0x01, 0x1d, 0xfa, 0x94, 0xc5, 0x03, 0xe2, 0x84, 0xb6, 0xba, - 0x30, 0xc9, 0x63, 0x44, 0x4f, 0xb5, 0x3c, 0x1c, 0x6f, 0xaa, 0xe1, 0xea, 0x9f, 0x72, 0xf0, 0xd6, - 0x99, 0xab, 0x8b, 0xb6, 0x79, 0x98, 0xb9, 0x39, 0xd3, 0x61, 0x7e, 0xfb, 0x8c, 0xb5, 0x9b, 0x8c, - 0xa7, 0x42, 0x89, 0x78, 0xa2, 0x5f, 0x1b, 0x80, 0x46, 0x44, 0x7d, 0xe2, 0xe8, 0x57, 0x24, 0xe7, - 0xfa, 0xfc, 0xdb, 0x49, 0x45, 0x73, 0x4b, 0x4f, 0xb0, 0xad, 0xf8, 0x5b, 0x11, 0x23, 0x27, 0xd6, - 0xb2, 0x3b, 0x2d, 0x47, 0x0c, 0x80, 0x60, 0x9a, 0x86, 0xd8, 0x4e, 0x62, 0x7d, 0x4b, 0x7b, 0xf0, - 0x2d, 0x59, 0x61, 0x09, 0xe2, 0xfd, 0x98, 0xca, 0xd9, 0x4b, 0x44, 0xf7, 0x51, 0x0b, 0xca, 0x7a, - 0x13, 0xda, 0x1e, 0x66, 0x8e, 0x1f, 0x50, 0x55, 0x02, 0x2f, 0x99, 0xf2, 0x33, 0x85, 0xa9, 0x3f, - 0x53, 0x98, 0x9b, 0xd1, 0x89, 0xbe, 0x28, 0x6b, 0x4c, 0x53, 0x42, 0xaa, 0x4d, 0xb8, 0x7c, 0xb6, - 0xa7, 0xfc, 0xda, 0xf5, 0x08, 0x9f, 0x88, 0xf4, 0x98, 0xb5, 0x78, 0x93, 0x5f, 0xcf, 0x9e, 0x38, - 0x41, 0x2a, 0xaf, 0x30, 0x59, 0x4b, 0x76, 0x36, 0xb2, 0x1f, 0x1a, 0xd5, 0x1f, 0xc3, 0xe2, 0xa4, - 0xa5, 0xaf, 0x43, 0xcf, 0x8c, 0xa3, 0x3f, 0x86, 0x82, 0xaa, 0x3a, 0xe8, 0x2a, 0x80, 0xb8, 0x5f, - 0xd8, 0x04, 0x3b, 0x32, 0x35, 0x67, 0xf4, 0xde, 0x15, 0x72, 0x0b, 0x3b, 0x1e, 0x7a, 0x07, 0x4a, - 0x22, 0x87, 0x85, 0x4e, 0x76, 0x4c, 0xa7, 0xc8, 0xc5, 0x5c, 0xa5, 0x51, 0x50, 0x93, 0xdd, 0xcb, - 0x15, 0xf3, 0xe5, 0x42, 0xed, 0xb7, 0x06, 0x54, 0x9a, 0x3e, 0x65, 0xdd, 0x8f, 0x77, 0x7e, 0x2a, - 0x3f, 0x93, 0xdc, 0x8d, 0x29, 0xf5, 0x13, 0x71, 0xcc, 0x5d, 0x9f, 0xfc, 0xa0, 0xb2, 0xd0, 0xb8, - 0xcc, 0xc9, 0xbe, 0x7e, 0xbe, 0xba, 0x38, 0x09, 0x19, 0x7d, 0x62, 0x69, 0xc3, 0xa5, 0xd0, 0x8f, - 0x6c, 0xc7, 0x75, 0x71, 0xc2, 0x53, 0x50, 0xc3, 0xb3, 0xaf, 0x84, 0xa3, 0xd0, 0x8f, 0x36, 0x15, - 0x44, 0xc9, 0x6a, 0xb7, 0xe0, 0xff, 0x94, 0x56, 0x93, 0x38, 0x7e, 0xe4, 0x47, 0x03, 0x61, 0xd2, - 0x1a, 0x14, 0x3d, 0xd5, 0x17, 0x36, 0xe9, 0xed, 0x36, 0x94, 0x36, 0x3e, 0x78, 0xfa, 0xcf, 0x95, - 0xcc, 0xd3, 0xd3, 0x15, 0xe3, 0xab, 0xd3, 0x15, 0xe3, 0xd9, 0xe9, 0x8a, 0xf1, 0x8f, 0xd3, 0x15, - 0xe3, 0x37, 0x2f, 0x56, 0x32, 0x5f, 0xbd, 0x58, 0xc9, 0x3c, 0x7b, 0xb1, 0x92, 0xf9, 0x74, 0x6e, - 0xec, 0x5b, 0xe1, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x7a, 0xbb, 0x53, 0xa9, 0x3d, 0x14, 0x00, - 0x00, +func init() { proto.RegisterFile("sql/execinfrapb/data.proto", fileDescriptor_data_607bf6adf2d0c389) } + +var fileDescriptor_data_607bf6adf2d0c389 = []byte{ + // 2083 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0x3d, 0x70, 0x1b, 0xc7, + 0x15, 0xc6, 0x01, 0x20, 0x7e, 0x1e, 0xff, 0xc0, 0x8d, 0x24, 0x23, 0x88, 0x42, 0xca, 0x50, 0xc6, + 0xa3, 0x38, 0xf6, 0x41, 0xa2, 0x0a, 0x39, 0x4c, 0x32, 0x12, 0x41, 0x80, 0x02, 0x65, 0x8a, 0xa4, + 0x0f, 0x54, 0x3c, 0xb2, 0xe3, 0xb9, 0x39, 0xdc, 0x2d, 0xc0, 0xb3, 0xee, 0x4f, 0xbb, 0x7b, 0x22, + 0xd9, 0x64, 0x32, 0x93, 0x94, 0x29, 0x52, 0x65, 0x52, 0x3a, 0x4d, 0x9a, 0x14, 0xe9, 0x53, 0xa5, + 0xca, 0xa8, 0xca, 0xb8, 0xd4, 0xa4, 0xd0, 0x24, 0x54, 0x93, 0x32, 0x6d, 0x5c, 0x65, 0xf6, 0x0f, + 0x7f, 0x22, 0x25, 0xd3, 0x72, 0x03, 0xec, 0xbe, 0x7d, 0xdf, 0xf7, 0xde, 0xbe, 0x7d, 0xfb, 0x76, + 0xf7, 0xa0, 0x46, 0x1f, 0x07, 0x0d, 0x7c, 0x84, 0x5d, 0x3f, 0xea, 0x13, 0x27, 0xe9, 0x35, 0x3c, + 0x87, 0x39, 0x66, 0x42, 0x62, 0x16, 0xa3, 0xaa, 0x1b, 0xbb, 0x8f, 0x48, 0xec, 0xb8, 0x07, 0x26, + 0x7d, 0x1c, 0x98, 0x9e, 0x4f, 0x19, 0x7d, 0x1c, 0x90, 0x34, 0xaa, 0x5d, 0xfa, 0x3c, 0xee, 0xd1, + 0x06, 0xff, 0x49, 0x7a, 0xe2, 0x4f, 0x22, 0x6a, 0x4b, 0x42, 0x3b, 0xe9, 0x35, 0x9c, 0xc4, 0x57, + 0x22, 0xa4, 0x45, 0x23, 0xe2, 0xda, 0x05, 0x2d, 0xc3, 0x84, 0xc4, 0x44, 0x83, 0x2f, 0xca, 0xde, + 0xb4, 0x78, 0x85, 0x7b, 0x98, 0x0c, 0x0e, 0x7d, 0x82, 0x1b, 0xc9, 0x40, 0x0c, 0x4d, 0x2a, 0x5c, + 0xe5, 0x0a, 0xae, 0xc3, 0x9c, 0x20, 0x1e, 0x34, 0x3c, 0x4c, 0xdd, 0xa4, 0xd7, 0xa0, 0x8c, 0xa4, + 0x2e, 0x4b, 0x09, 0xf6, 0x94, 0xd2, 0x3b, 0xa7, 0x28, 0xe1, 0xc8, 0x8d, 0x3d, 0xec, 0xd9, 0x9e, + 0xc3, 0xd2, 0x50, 0x3b, 0xc1, 0xf5, 0xd8, 0x71, 0x82, 0xa9, 0xfc, 0x55, 0xe2, 0x77, 0x53, 0xe6, + 0x07, 0x0d, 0x46, 0x1c, 0xd7, 0x8f, 0x06, 0xfa, 0x3f, 0xe9, 0x35, 0x08, 0x76, 0x63, 0xc2, 0x39, + 0x68, 0xe2, 0x44, 0x7a, 0x76, 0x83, 0x78, 0x10, 0x8b, 0x66, 0x83, 0xb7, 0x94, 0xb4, 0x2a, 0x18, + 0x0e, 0x02, 0xb7, 0xc1, 0xfc, 0x10, 0x53, 0xe6, 0x84, 0x89, 0x1a, 0xf9, 0xee, 0x20, 0x8e, 0x07, + 0x01, 0x6e, 0x88, 0x5e, 0x2f, 0xed, 0x37, 0x9c, 0xe8, 0x58, 0x0e, 0xd5, 0x3f, 0x81, 0x99, 0x36, + 0x9f, 0x2a, 0xba, 0x0d, 0xd0, 0x4f, 0x83, 0xc0, 0x16, 0x13, 0xaf, 0xe6, 0xae, 0x18, 0xd7, 0x66, + 0x57, 0xaf, 0x98, 0xa3, 0xf5, 0xd1, 0xa1, 0x33, 0xdb, 0x72, 0x4e, 0x02, 0x65, 0x95, 0x39, 0x46, + 0x34, 0xd7, 0xf2, 0x7f, 0xf8, 0x62, 0x25, 0x73, 0x2f, 0x5f, 0x32, 0x2a, 0xd9, 0x7b, 0xf9, 0x52, + 0xb6, 0x92, 0xab, 0xef, 0x01, 0xb4, 0x8f, 0x12, 0x82, 0x29, 0xf5, 0xe3, 0x08, 0x2d, 0x43, 0xf1, + 0x09, 0x26, 0xbc, 0x59, 0x35, 0xae, 0x18, 0xd7, 0xca, 0xcd, 0xfc, 0xd3, 0xe7, 0x2b, 0x19, 0x4b, + 0x0b, 0x51, 0x15, 0xf2, 0xf8, 0x28, 0x21, 0xd5, 0xec, 0xd8, 0xa0, 0x90, 0xac, 0x95, 0x38, 0xf3, + 0x7f, 0xff, 0xb8, 0x92, 0xa9, 0xff, 0x26, 0x0b, 0xa5, 0x5d, 0xe2, 0x61, 0xe2, 0x47, 0x03, 0xb4, + 0x05, 0x45, 0x37, 0x0e, 0xd2, 0x30, 0xa2, 0x55, 0xe3, 0x4a, 0xee, 0xda, 0xec, 0xea, 0x0f, 0xcd, + 0xb3, 0xd2, 0xc9, 0xd4, 0x20, 0x73, 0x43, 0x20, 0xb4, 0x6d, 0x85, 0xaf, 0xfd, 0xc9, 0x80, 0x82, + 0x1c, 0x41, 0xdf, 0x17, 0xac, 0xb6, 0xef, 0x1d, 0x09, 0x37, 0xe7, 0x95, 0x6a, 0xc1, 0x8d, 0x83, + 0x2d, 0xef, 0x08, 0x7d, 0x0c, 0x65, 0xcf, 0x27, 0xd8, 0x65, 0x7c, 0x1e, 0xdc, 0xd5, 0x85, 0xd5, + 0x9b, 0x5f, 0xdb, 0xac, 0xd9, 0xd2, 0x50, 0xc5, 0x3a, 0xe2, 0xaa, 0x2f, 0x43, 0x79, 0x38, 0x8a, + 0x8a, 0x90, 0x5b, 0xef, 0x6e, 0x54, 0x32, 0xa8, 0x04, 0xf9, 0x56, 0xbb, 0xbb, 0x51, 0x31, 0xd6, + 0xf2, 0xff, 0xf9, 0x62, 0x45, 0xfd, 0xd6, 0xff, 0x9a, 0x03, 0xd4, 0x65, 0x04, 0x3b, 0x61, 0x3b, + 0xf2, 0x92, 0xd8, 0x8f, 0x58, 0x37, 0xc1, 0x2e, 0xfa, 0x10, 0xf2, 0x3c, 0xa3, 0x84, 0xdf, 0x0b, + 0xab, 0x37, 0xce, 0x76, 0xeb, 0x65, 0xac, 0xb9, 0x7f, 0x9c, 0x60, 0x1d, 0x74, 0x4e, 0x82, 0x7e, + 0x0c, 0x65, 0x2a, 0xd4, 0x6c, 0xdf, 0x13, 0x13, 0x9d, 0x69, 0x5e, 0xe6, 0xc3, 0x27, 0xcf, 0x57, + 0x4a, 0x12, 0xbf, 0xd5, 0xfa, 0x6a, 0xac, 0x6d, 0x95, 0xa4, 0xfa, 0x96, 0x87, 0x1e, 0xc3, 0x02, + 0x73, 0xc8, 0x00, 0x33, 0x3b, 0x8a, 0x3d, 0xcc, 0xf1, 0x79, 0x81, 0xff, 0x50, 0xe1, 0xe7, 0xf6, + 0xc5, 0xe8, 0x4e, 0xec, 0x61, 0xc1, 0x71, 0x73, 0xe0, 0xb3, 0x83, 0xb4, 0x67, 0xba, 0x71, 0xd8, + 0x18, 0xfa, 0xec, 0xf5, 0x46, 0xed, 0x46, 0xf2, 0x68, 0xd0, 0x50, 0x3b, 0xda, 0x94, 0x30, 0x6b, + 0x8e, 0x8d, 0x48, 0x84, 0xc9, 0x98, 0xf8, 0x03, 0x3f, 0x1a, 0x9a, 0x9c, 0x99, 0x34, 0xb9, 0x2b, + 0x46, 0xdf, 0xd0, 0x64, 0x3c, 0x22, 0xf1, 0xea, 0xd7, 0x21, 0xcf, 0x83, 0x86, 0xca, 0x30, 0xb3, + 0xbd, 0xbb, 0xb1, 0xbe, 0x5d, 0xc9, 0x20, 0x80, 0x82, 0xd5, 0xbe, 0xbf, 0xbb, 0xdf, 0xae, 0x18, + 0x68, 0x09, 0xe6, 0xbb, 0x0f, 0x77, 0x36, 0x6c, 0xab, 0xdd, 0xdd, 0xdb, 0xdd, 0xe9, 0xb6, 0xc5, + 0xae, 0xc8, 0x55, 0xf2, 0xf5, 0x7f, 0x64, 0x61, 0x7e, 0x2b, 0x4a, 0x52, 0xd6, 0x3d, 0x8e, 0x5c, + 0xb1, 0x6e, 0x9b, 0x13, 0xeb, 0xf6, 0xde, 0xd9, 0xeb, 0x36, 0x01, 0x7b, 0x79, 0xc9, 0x5a, 0x50, + 0x8a, 0x55, 0xc2, 0x89, 0x15, 0x9b, 0x5d, 0xad, 0xbf, 0x3e, 0x35, 0x15, 0xc3, 0x10, 0x89, 0xb6, + 0xa1, 0x28, 0x57, 0x92, 0x56, 0x73, 0x62, 0x5b, 0xbd, 0x77, 0x9e, 0x44, 0xd2, 0x3b, 0x4b, 0x51, + 0xa0, 0x9f, 0xc1, 0x9c, 0xdc, 0x64, 0xb6, 0x28, 0x76, 0xd5, 0xbc, 0xa0, 0xac, 0x4d, 0x51, 0x52, + 0x1c, 0x9a, 0xb2, 0x18, 0xee, 0x5b, 0xb3, 0x52, 0x9f, 0xcf, 0x8e, 0xd6, 0xeb, 0x2a, 0xc8, 0xf3, + 0x50, 0x7e, 0xb0, 0xb3, 0x6b, 0xb5, 0xda, 0x56, 0xbb, 0x55, 0xc9, 0xa0, 0x59, 0x28, 0xea, 0x8e, + 0x51, 0xff, 0x5f, 0x01, 0x2a, 0xbb, 0x29, 0x4b, 0x52, 0x66, 0xc5, 0x29, 0xc3, 0x44, 0xc4, 0x74, + 0x6b, 0x22, 0xa6, 0x8d, 0x57, 0xc4, 0x61, 0x0a, 0xf9, 0x72, 0x58, 0xc7, 0x02, 0x92, 0x7d, 0xf3, + 0x80, 0xbc, 0x0d, 0x73, 0x07, 0x0e, 0x3d, 0xb0, 0x75, 0xe9, 0xe2, 0x31, 0x9e, 0xb7, 0x66, 0xb9, + 0x4c, 0x16, 0x09, 0x8a, 0x02, 0x58, 0x22, 0x4e, 0x34, 0xc0, 0x36, 0x11, 0x5e, 0xd9, 0x34, 0xc1, + 0xae, 0xd8, 0x42, 0xb3, 0xab, 0x6b, 0xe7, 0x98, 0x88, 0xc5, 0x39, 0x46, 0x7d, 0xe5, 0xc8, 0x22, + 0x99, 0x14, 0xa3, 0x1b, 0xb0, 0xe4, 0xf9, 0xd4, 0xe9, 0x05, 0xd8, 0xee, 0xa5, 0xfd, 0xbe, 0x4c, + 0x1f, 0xbe, 0x7b, 0x4a, 0x0a, 0x51, 0x51, 0xc3, 0x4d, 0x3d, 0x5a, 0xfb, 0x7b, 0x0e, 0x16, 0xa7, + 0xd8, 0xd1, 0xa7, 0x30, 0xc3, 0x4f, 0x28, 0x5d, 0x8b, 0x6f, 0x7f, 0x73, 0x47, 0xcd, 0x6e, 0xe2, + 0xe8, 0x02, 0x29, 0x39, 0x79, 0xd0, 0x3c, 0xdc, 0x77, 0xd2, 0x80, 0xd9, 0x1e, 0xa6, 0x4c, 0xd6, + 0x23, 0x6b, 0x56, 0xc9, 0x5a, 0x98, 0x32, 0x14, 0x42, 0x59, 0x9c, 0xb6, 0x7e, 0x34, 0xd0, 0x89, + 0xbb, 0xf5, 0x06, 0x3e, 0xc8, 0xb5, 0x68, 0x2b, 0x46, 0x5d, 0xae, 0x87, 0x16, 0x6a, 0x4f, 0x60, + 0x61, 0x52, 0x05, 0x5d, 0x86, 0x82, 0x5c, 0xd3, 0x97, 0xce, 0x0d, 0x7e, 0xac, 0x6c, 0x42, 0x49, + 0x83, 0xd5, 0xb1, 0xf1, 0x83, 0xe9, 0x3d, 0xf0, 0x38, 0xe8, 0x39, 0x14, 0x9b, 0x2d, 0x7e, 0x57, + 0x98, 0x32, 0x3c, 0xc4, 0xd6, 0xb6, 0x21, 0xcf, 0xc3, 0x83, 0x2e, 0xc0, 0x0c, 0x65, 0x0e, 0x61, + 0xc2, 0xd8, 0x9c, 0x25, 0x3b, 0xa8, 0x02, 0x39, 0x1c, 0xc9, 0x72, 0x3d, 0x67, 0xf1, 0x26, 0xf7, + 0x4a, 0x66, 0x9e, 0x38, 0xd2, 0x67, 0xb4, 0x57, 0x52, 0x56, 0xbf, 0xad, 0xb6, 0x57, 0x05, 0xe6, + 0xf6, 0xd6, 0xbb, 0x5d, 0x7b, 0xbf, 0x63, 0xed, 0x3e, 0xb8, 0xdb, 0x91, 0xa5, 0xec, 0xfe, 0x96, + 0x65, 0xed, 0x5a, 0x15, 0x83, 0xef, 0xb6, 0xe6, 0x43, 0xbb, 0xb3, 0xde, 0xed, 0x54, 0xb2, 0x68, + 0x0e, 0x4a, 0xcd, 0x87, 0xb6, 0xb5, 0xbe, 0x73, 0xb7, 0x5d, 0xc9, 0xd5, 0x7f, 0x6d, 0x40, 0x59, + 0x38, 0xbc, 0x15, 0xf5, 0xe3, 0x89, 0x49, 0x1a, 0xdf, 0x7c, 0x92, 0xc8, 0x54, 0x9b, 0x57, 0x16, + 0xb1, 0x57, 0x15, 0x0b, 0xa1, 0x57, 0xff, 0x25, 0x2c, 0xec, 0x91, 0xd8, 0x4b, 0x5d, 0x4c, 0x3a, + 0xd8, 0xf1, 0x30, 0x41, 0x37, 0xa0, 0xd8, 0x0f, 0xe2, 0x43, 0x7e, 0x10, 0x88, 0x00, 0x35, 0xab, + 0xdc, 0xc4, 0x3f, 0x9f, 0xaf, 0x14, 0x36, 0x83, 0xf8, 0x70, 0xab, 0x75, 0x32, 0x6c, 0x59, 0x05, + 0xae, 0xb8, 0xe5, 0xbd, 0xc1, 0x81, 0x57, 0xff, 0xb3, 0x01, 0x73, 0xda, 0x81, 0x96, 0xc3, 0x1c, + 0xf4, 0x3d, 0x28, 0x13, 0xe7, 0xd0, 0xee, 0x1d, 0x33, 0x4c, 0xd5, 0x0a, 0x95, 0x88, 0x73, 0xd8, + 0xe4, 0x7d, 0x64, 0x41, 0x29, 0xc4, 0xcc, 0xe1, 0xb7, 0x55, 0x55, 0x50, 0xae, 0x9f, 0x9d, 0xa8, + 0x16, 0x0e, 0x63, 0x86, 0x35, 0xf9, 0x7d, 0x85, 0xd3, 0x11, 0xd3, 0x3c, 0xe8, 0x5d, 0x58, 0x88, + 0xd2, 0xd0, 0xc6, 0x61, 0xc2, 0x8e, 0x6d, 0x12, 0x1f, 0xd2, 0x89, 0xe5, 0x9e, 0x8b, 0xd2, 0xb0, + 0xcd, 0x87, 0xac, 0xf8, 0x90, 0xd6, 0x9f, 0x19, 0xb0, 0x38, 0x22, 0xa4, 0xd4, 0x19, 0x60, 0x74, + 0x07, 0x0a, 0x07, 0x22, 0x72, 0xc2, 0xdb, 0xd9, 0xd5, 0x6b, 0x67, 0x7b, 0x34, 0x19, 0x69, 0x4b, + 0xe1, 0xd0, 0x3a, 0x14, 0xd8, 0x71, 0x22, 0xd3, 0x9b, 0xcf, 0xe9, 0xea, 0xd9, 0x0c, 0xc3, 0x84, + 0xd1, 0xd9, 0x28, 0x81, 0xe8, 0x0e, 0xe4, 0x45, 0x50, 0xe4, 0xe5, 0xf3, 0x9d, 0xd7, 0xbb, 0xd0, + 0x1a, 0x85, 0x42, 0x20, 0xeb, 0x7f, 0x5b, 0x80, 0x4b, 0xa7, 0x47, 0x0c, 0xfd, 0x02, 0x40, 0x16, + 0x55, 0x3f, 0xea, 0xc7, 0x6a, 0x96, 0x3f, 0x39, 0x6f, 0xdc, 0x65, 0x99, 0xe0, 0xae, 0xd3, 0x4e, + 0xc6, 0x2a, 0x13, 0xdd, 0x43, 0xb7, 0x60, 0x46, 0x5e, 0x9c, 0x65, 0xca, 0xae, 0x9c, 0x4d, 0x2c, + 0x2e, 0xcb, 0x9d, 0x8c, 0x25, 0xf5, 0xd1, 0xa7, 0x00, 0xfc, 0xae, 0x8f, 0xed, 0xb1, 0x99, 0xaf, + 0x9d, 0xdb, 0xad, 0x7d, 0x4e, 0xc1, 0xa3, 0xc1, 0xbd, 0x62, 0xba, 0x83, 0x3e, 0x86, 0x0b, 0x01, + 0x76, 0xfa, 0x36, 0x3b, 0x8a, 0xec, 0xbe, 0x1f, 0x39, 0x81, 0x4d, 0x99, 0xc3, 0xb0, 0x3a, 0x4b, + 0xc6, 0xf7, 0xa6, 0xbe, 0xe9, 0x6c, 0x63, 0xa7, 0xbf, 0x7f, 0x14, 0x6d, 0x72, 0xe5, 0x2e, 0xd7, + 0xed, 0x64, 0xac, 0xa5, 0x60, 0x5a, 0x88, 0x2c, 0x28, 0x92, 0xf8, 0xd0, 0x8e, 0xd2, 0x50, 0x9c, + 0x14, 0xb3, 0xab, 0xb7, 0xce, 0x1f, 0xc9, 0xf8, 0x70, 0x27, 0x0d, 0x3b, 0x19, 0xab, 0x40, 0x44, + 0x0b, 0x85, 0x50, 0xa1, 0x4e, 0x98, 0x04, 0x98, 0xd8, 0x09, 0x89, 0x07, 0xfc, 0xd9, 0x50, 0x2d, + 0x0a, 0xf2, 0x3b, 0xe7, 0x26, 0xef, 0x4a, 0xa2, 0x3d, 0xc5, 0xd3, 0xc9, 0x58, 0x8b, 0x74, 0x52, + 0x84, 0xf6, 0xa1, 0x18, 0x62, 0x46, 0x7c, 0x97, 0x56, 0x4b, 0xc2, 0xca, 0x07, 0xe7, 0xb6, 0x72, + 0x5f, 0xe2, 0x3b, 0x19, 0x4b, 0x53, 0xa1, 0x5f, 0x19, 0xf0, 0x56, 0x2f, 0x0d, 0x1e, 0xf1, 0x29, + 0xb8, 0x98, 0xd2, 0x78, 0x6c, 0x32, 0x65, 0x61, 0x66, 0xf3, 0xdc, 0x66, 0x9a, 0x69, 0xf0, 0x68, + 0x4f, 0xd3, 0x8d, 0x4d, 0xe9, 0x62, 0xef, 0xb4, 0x81, 0xda, 0x2e, 0xc0, 0x28, 0x4b, 0xd1, 0xfa, + 0x54, 0xda, 0xf3, 0xad, 0x79, 0xf9, 0x94, 0x85, 0x1f, 0x42, 0xf4, 0x51, 0x37, 0xcc, 0xed, 0xda, + 0xe7, 0x50, 0x1e, 0xe6, 0x17, 0xfa, 0x0c, 0x16, 0xdd, 0x38, 0x08, 0xb0, 0xcb, 0xd4, 0x93, 0x54, + 0x1f, 0xf8, 0xe6, 0x18, 0x29, 0x7f, 0x88, 0x9a, 0xea, 0x09, 0x6b, 0x0e, 0x9f, 0xb2, 0xa6, 0xa5, + 0x9e, 0xb2, 0x63, 0xe7, 0xfb, 0xc2, 0x90, 0x8c, 0x0b, 0x69, 0xed, 0x10, 0x0a, 0x32, 0x31, 0xd0, + 0xfb, 0x50, 0xa6, 0x38, 0xf2, 0x30, 0xd1, 0x35, 0xbc, 0xdc, 0xac, 0x0c, 0xcb, 0xb1, 0x18, 0x10, + 0x25, 0x58, 0xb6, 0x3c, 0xfe, 0x6c, 0xd3, 0x19, 0x99, 0x1d, 0x3f, 0xe8, 0x54, 0x72, 0xad, 0x40, + 0x29, 0x70, 0x28, 0xb3, 0x43, 0x3a, 0x10, 0x9b, 0x4c, 0xdf, 0x6d, 0x8a, 0x5c, 0x7a, 0x9f, 0x0e, + 0x6a, 0x14, 0x16, 0xa7, 0x92, 0x06, 0xfd, 0x08, 0x16, 0x78, 0x25, 0xd5, 0x4b, 0x89, 0xa5, 0x1b, + 0x79, 0x85, 0x9c, 0xe7, 0x63, 0x7b, 0x7a, 0x08, 0xdd, 0x04, 0x74, 0xe0, 0x53, 0x16, 0x0f, 0x88, + 0x13, 0xda, 0xea, 0xc2, 0x24, 0x8f, 0x11, 0x6d, 0x6a, 0x69, 0x38, 0xde, 0x52, 0xc3, 0xb5, 0xbf, + 0xe4, 0xe1, 0xe2, 0xa9, 0xab, 0x8b, 0x36, 0x79, 0x98, 0xb9, 0x3b, 0xd3, 0x61, 0x7e, 0xeb, 0x94, + 0xb5, 0x9b, 0x8c, 0xa7, 0x42, 0x89, 0x78, 0xa2, 0xdf, 0x1a, 0x80, 0x46, 0x44, 0x7d, 0xe2, 0xe8, + 0x87, 0x2b, 0xe7, 0xfa, 0xec, 0xdb, 0x49, 0x45, 0x73, 0x43, 0x1b, 0xd8, 0x54, 0xfc, 0xed, 0x88, + 0x91, 0x63, 0x6b, 0xc9, 0x9d, 0x96, 0x23, 0x06, 0x40, 0x30, 0x4d, 0x43, 0x6c, 0x27, 0xb1, 0xbe, + 0xa5, 0x3d, 0xf8, 0x96, 0xbc, 0xb0, 0x04, 0xf1, 0x5e, 0x4c, 0xa5, 0xf5, 0x32, 0xd1, 0x7d, 0xd4, + 0x86, 0x8a, 0xde, 0x84, 0xb6, 0x87, 0x99, 0xe3, 0x07, 0x54, 0x95, 0xc0, 0x0b, 0xa6, 0xfc, 0x32, + 0x62, 0xea, 0x2f, 0x23, 0xe6, 0x7a, 0x74, 0xac, 0x2f, 0xca, 0x1a, 0xd3, 0x92, 0x90, 0x5a, 0x0b, + 0x2e, 0x9d, 0x3e, 0x53, 0x7e, 0xed, 0x7a, 0x84, 0x8f, 0x45, 0x7a, 0xcc, 0x58, 0xbc, 0xc9, 0xaf, + 0x67, 0x4f, 0x9c, 0x20, 0x95, 0x57, 0x98, 0xac, 0x25, 0x3b, 0x6b, 0xd9, 0x0f, 0x8c, 0xda, 0x4f, + 0x61, 0x61, 0xd2, 0xd3, 0xd7, 0xa1, 0x73, 0xe3, 0xe8, 0x8f, 0xa0, 0xa8, 0xaa, 0x0e, 0xba, 0x0a, + 0x20, 0xee, 0x17, 0x36, 0xc1, 0x8e, 0x4c, 0xcd, 0x9c, 0xde, 0xbb, 0x42, 0x6e, 0x61, 0xc7, 0x43, + 0x6f, 0x43, 0x59, 0xe4, 0xb0, 0xd0, 0xc9, 0x8e, 0xe9, 0x94, 0xb8, 0x98, 0xab, 0x34, 0x8b, 0xca, + 0xd8, 0xbd, 0x7c, 0xa9, 0x50, 0x29, 0xd6, 0x7f, 0x6f, 0x40, 0xb5, 0xe5, 0x53, 0xd6, 0xfd, 0x68, + 0xfb, 0xe7, 0xf2, 0xcb, 0xcc, 0xdd, 0x98, 0x52, 0x3f, 0x11, 0xc7, 0xdc, 0xf5, 0xc9, 0x6f, 0x38, + 0xf3, 0xcd, 0x4b, 0x9c, 0xec, 0xab, 0xe7, 0x2b, 0x0b, 0x93, 0x90, 0xd1, 0x57, 0x9d, 0x0e, 0x5c, + 0x08, 0xfd, 0xc8, 0x76, 0x5c, 0x17, 0x27, 0x3c, 0x05, 0x35, 0x3c, 0xfb, 0x4a, 0x38, 0x0a, 0xfd, + 0x68, 0x5d, 0x41, 0x94, 0xac, 0x7e, 0x0b, 0xbe, 0xa3, 0xb4, 0x5a, 0xc4, 0xf1, 0x23, 0x3f, 0x1a, + 0x08, 0x97, 0xae, 0x40, 0xc9, 0x53, 0x7d, 0xe1, 0x93, 0xde, 0x6e, 0x43, 0x69, 0xf3, 0xfd, 0xa7, + 0xff, 0x5e, 0xce, 0x3c, 0x3d, 0x59, 0x36, 0xbe, 0x3c, 0x59, 0x36, 0x9e, 0x9d, 0x2c, 0x1b, 0xff, + 0x3a, 0x59, 0x36, 0x7e, 0xf7, 0x62, 0x39, 0xf3, 0xe5, 0x8b, 0xe5, 0xcc, 0xb3, 0x17, 0xcb, 0x99, + 0x4f, 0x66, 0xc7, 0x3e, 0x4f, 0xfe, 0x3f, 0x00, 0x00, 0xff, 0xff, 0x2f, 0x15, 0xd5, 0x60, 0xb0, + 0x14, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/data.proto b/pkg/sql/execinfrapb/data.proto index fd6c38338c73..9af0c13b63f2 100644 --- a/pkg/sql/execinfrapb/data.proto +++ b/pkg/sql/execinfrapb/data.proto @@ -108,6 +108,10 @@ message StreamEndpointSpec { optional int32 target_node_id = 4 [(gogoproto.nullable) = false, (gogoproto.customname) = "TargetNodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; + // Node ID of the origin node, only used for REMOTE streams. + optional int32 origin_node_id = 5 [(gogoproto.nullable) = false, + (gogoproto.customname) = "OriginNodeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; reserved 3; } diff --git a/pkg/sql/execstats/traceanalyzer.go b/pkg/sql/execstats/traceanalyzer.go index 42481fccb632..a8db914050e1 100644 --- a/pkg/sql/execstats/traceanalyzer.go +++ b/pkg/sql/execstats/traceanalyzer.go @@ -132,10 +132,10 @@ func getNetworkBytesFromDistSQLSpanStats(dss execinfrapb.DistSQLSpanStats) (int6 switch v := dss.(type) { case *flowinfra.OutboxStats: return v.BytesSent, nil - case *execpb.VectorizedStats: - // VectorizedStats are output by the Inbox, hence the read/sent difference + case *execpb.VectorizedInboxStats: + // VectorizedInboxStats are output by the Inbox, hence the read/sent difference // with OutboxStats. - return v.BytesRead, nil + return v.BaseVectorizedStats.BytesRead, nil } return 0, errors.Errorf("could not get network bytes from %T", dss) } diff --git a/pkg/sql/logictest/testdata/logic_test/dist_vectorize b/pkg/sql/logictest/testdata/logic_test/dist_vectorize index e2a6880cb96b..f20c8bbab71d 100644 --- a/pkg/sql/logictest/testdata/logic_test/dist_vectorize +++ b/pkg/sql/logictest/testdata/logic_test/dist_vectorize @@ -51,12 +51,12 @@ NULL /1 {5} 5 query T SELECT url FROM [EXPLAIN ANALYZE SELECT count(*) FROM kv] ---- -https://cockroachdb.github.io/distsqlplan/decode.html#eJzMlV1vm0wQhe_fX7GaK_vVWpgPOw5XcVNXQnIgNY76EaGIwMhFwSzdXdJElv97BaQqWM1CLxpxybCHOYdnYA4gvqdgg79ary63pOAp-bDxrsjt6vP1eum4ZOku11--rsjoveNv_Y_rMXk5GrEik6P_x_X5h8cAKGQsRjfcowD7FnSgYAAFEyhYQGEGAYWcswiFYLw8cqgETvwE9pRCkuWFLMsBhYhxBPsAMpEpgg3b8D7FDYYxcm0KFGKUYZJWbR4eL3Ke7EP-DBT8PMyETSZa2dgrpE1cliFQuA9l9A0FYYXMy3L5EFnkaaNU-nU8IpM92mQqStGzREE4hrFNFuQdUODsx6-CDsGRQi1-cS1kuEOw9SPtn2y523HchZJxbdYOdunduNu7jffJH437J8AnjAqZsEwV5DXnxqvOfxsuMsZj5Bi33AZHdTb9hJp_c3XnuNvRhf5W2cxWNr3_vOnd86YZE80czMR1ZGtQmQ9s4oz-VIweVMyJZg2GSke2BpWzgVEx-1Mxe1CxJtWfbhhUOrI1qCwGRsXqT8XqQWU2GQyTjmQNJucDY9Kx6zcocpYJPNmcf37ytNyoGO-wXr-CFTzCa86iqk196VW6qhCjkPVdvb5wsvpWabAp1pVioyXWT8WGunNHa1OpttRi6298V2-xeqH_ao7b5mZKc3N1svmQk50pzS3UyRZDTnaunuVpx2ek_gjfOltw_O9nAAAA__--x3E3 +https://cockroachdb.github.io/distsqlplan/decode.html#eJzMlU9vm0AQxe_9FKs5OdVamD92HE5xU1dCciA1jvonQhGBkYuCWbq7pIksf_cKSFWwkoUcWnFk2Me8x29g9iB-pmCDv1wtLzak4Cn5tPYuyc3y69Vq4bhk4S5W374vyeij42_8z6sT8nw0YkUmR-9P6vP3DwFQyFiMbrhDAfYN6EDBAAomULCAwhQCCjlnEQrBeHlkXwmc-BHsCYUkywtZlgMKEeMI9h5kIlMEGzbhXYprDGPk2gQoxCjDJK3a3D-c5zzZhfwJKPh5mAmbjLWysVdIm7gsQ6BwF8roBwrCCpmX5fIhssjTRqn063hEJju0yUSUoieJgnAMY5vMyQegwNmvPwUdggOFWvzsWshwi2DrB9o_2WK75bgNJePatB3swrt2N7dr74s_OumfAB8xKmTCMlWQ15wbrzr_a7jIGI-RY9xyGxzU2fQjav715a3jbkbn-v_KZray6f3nTe-eN80Ya-ZgJq4jW4PKbGATZ_SnYvSgYo41azBUOrI1qJwOjIrZn4rZg4o1rv50w6DSka1BZT4wKlZ_KlYPKtPxYJh0JGswORsYk45dv0aRs0zg0eZ8-cmTcqNivMV6_QpW8AivOIuqNvWlV-mqQoxC1nf1-sLJ6lulwaZYV4qNllg_Fhvqzh2tTaXaUoutt_iu3mL1Ql-cgrcObdvJVOlkpo4xG0yMU6WTuTrGfDAxztQjOen4GtTf0j8NEhze_Q4AAP__MqxXkw== query T SELECT url FROM [EXPLAIN ANALYZE SELECT * FROM kv JOIN kw ON kv.k = kw.k] ---- -https://cockroachdb.github.io/distsqlplan/decode.html#eJzUWF1P40YUfe-vGN2n3e5k7fEHC5ZWCm2plBWbbIGHtqs8mHhKrATbHY9hEeK_V46LICEZj3MzjvNG7Bmfe67POYzvI-T_ziGAy7Pzs1-vSCHm5PeL0Vfy_ezPb-engyE5HZ6e__X3GXn32-Dy6vKP8_fk_6U_Vwtnd-TLaDAks3syGpLZ3ccZ-Uxm9x9nY6CQpBEfhrc8h-A7MKDgAAUXKHhAwYcxhUykE57nqSiXPC42DKIfENgU4iQrZHl5TGGSCg7BI8hYzjkEcBVez_kFDyMuLBsoRFyG8XwBM7vrZyK-DcUDULjMwiQPSM8qga9DOZnynKSFzAoZkHKjLLL5q0tljYMRkfEtD4idl5seJM-J4GEUkGPyC1AQ6f3zBQbjJwrV5orAc4HXD2Qa5tPl0vrl-jGFXIY3HAL2RLdjfLTC-P4tY4t1k7OzkfPLc1IRccGj1ed8KIG1Vq1p31cubviXNE64sNiKYub8H_muzz68_yzim2n1J1AYlc3pM9p3aN-lfU-_m_wHnxQyThNVU5e7-NIhV6NDRbKO_Vriw7SXZhbzlylrEPF1iHi2gom3xITp65vVO9pyepbbQX2zbTl_0vB0Rzk7Gzm36Gl2OJ529FXhaDjB7VkNeOzJCQ04H2s4oaOcnY2cW3SCczhOcPVV4Wo4wetZfgdVwbblfKLhhI5ydjZybtEJ7uE4wdNXhafhBL_XQU2wbRn79T7oWYyESUQYSeWUiw6ydzayb9ER3uE4ouZ7-ILnWZrkfOULaP2T7bIxPLrhVbvztBAT_k2kkwVM9XO02Lc4v0Y8l9Vdp_oxSJ5v5TKUul9Q9vqGLN6lfglH2BJeK7wh9gkGext3NSmOoV8OojPMNwrOGqjSMaNKdQlH2BIQ9E8w2HhVqotj6JeD6MyKKncN7qyC26_B3WVHrG52lZvZsqBsM5r2zIa9snlqbHTK4-mjwh5J33CW14CjshwfKL7ZsFd2Ro2NTnkdVapLQIU9kr7hLK8BR2U5XpVHTcK-7eI-7THI1dhmj-tqbHSC69hVXQIuyE0Lw_Cp_XiPQa7GNntcV2OjE1xHleoScEGOV2VNdWZP7SddDnJmeP6i_pgyOHkx3hqzx_UacPx5XWsEsd8ZjOEhDAYclec7kKbBAQ22NfhDu5Y09zqIYY0mMa2r482oZ3eZbrw6s-f3GnDDma4GNzyDqUNHh7mWbZtMANuP1SYzwp1nSpPpYMvgpofqjeaDpv6jvBkH7TXTx08__RcAAP__bh716Q== +https://cockroachdb.github.io/distsqlplan/decode.html#eJzUWF1v2zYUfd-vIO5Tu9KVqI80EVDA2ZYBLlK7S_KwrfCDYnG2YEfSKCpuEOS_D7IWNHZsiqQoVX5LZFLnnqtzDnj5CPm_Kwjg-uLy4tcbVLAV-v1q8hl9vfjzy-X5aIzOx-eXf_19gd78Nrq-uf7j8i36f-nP1cLlPfo0GY3Rco0mY7S8f79EH9Fy_X45BQxJGtFxeEdzCL4CAQwOYHABgwcYfJhiyFg6o3mesnLJ42bDKPoGgY0hTrKCl4-nGGYpoxA8Ao_5ikIAN-Htil7RMKLMsgFDRHkYrzYwy_thxuK7kD0AhussTPIADawS-DbkswXNUVrwrOABKjfyIlu9eFTWOJogHt_RANl5uemB0xwxGkYBOkW_AAaWrp8fEJg-Yag2VwSeC7x9QIswX2yXNizXTzHkPJxTCMgT1mN8ssN4_ZqxRfrJ2TnI-ft7UhZRRqPd97wrgaVW7WnfZ8rm9FMaJ5RZZEcxK_oPfzMk795-ZPF8Uf0JGCZlc4YEDx08dPHQk-8m_UZnBY_TRNTU7S5-75Ar0aEi2cd-L_FxOkgzi_jblCWI-DJEPFvAxNtiQuT1TeodbTkDy-2hvoku5w8Snu4pZ-cg5w49TY7H0468KhwJJ7gDS4HHD3KCAudTCSf0lLNzkHOHTnCOxwmuvCpcCSd4A8vvoSqILuczCSf0lLNzkHOHTnCPxwmevCo8CSf4gx5qgugy9ut9MLAICpMIEZTyBWU9ZO8cZN-hI7zjcUTNPHxF8yxNcrozAe1_s102hkZzWrU7Tws2o19YOtvAVP9ONvs259eI5rz61an-GSXPP-U85LITlL2_IZtvKV_CiVIJL-WsCHSmDSTjG5VKiFrbG3AmvjkkoqAkpx0liUs4USqhAdczbSB1JYkrIWptb8B5R0mNkJxdJPslkrst2d3NrnAz2RaB3Y4OPYOhKuyUGEgtOlvhqh-qikAmM7MGST8z1e3tGwxVIWcxkFp0aipJXIJ-qCoCmczMGiT9zFRX0olKqLZayYeu0lEMZPBgKQZSy0BN84hLaJCOpr-vySPnaVfpKAYyeLAUA6lloKaSxCU0SEd1JdWUYvDIedabdCQmJ3Lxgb6twds4aYOnyBokxaDUnVM7nMpNjuVNkPRDUkNObc3nqkiKaakrp-5Gc6I0m7f7kV9N-oaC0ngpBk-aNUgmg1KMZHLeroNSi0NdE6lc2rQcXyrXOs3srHJ70yaS0YtKpfubtjL51ejfXVBOn376LwAA__9Jy9vs # Verify execution. statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/explain_analyze_plans b/pkg/sql/logictest/testdata/logic_test/explain_analyze_plans index 3d26e5386512..035050353f19 100644 --- a/pkg/sql/logictest/testdata/logic_test/explain_analyze_plans +++ b/pkg/sql/logictest/testdata/logic_test/explain_analyze_plans @@ -60,13 +60,13 @@ NULL /1 {5} 5 query T SELECT url FROM [EXPLAIN ANALYZE (DISTSQL) SELECT kv.k, avg(kw.k) FROM kv JOIN kw ON kv.k=kw.k GROUP BY kv.k] ---- -https://cockroachdb.github.io/distsqlplan/decode.html#eJzcWWFP6kgU_b6_YjKffHnllZkWRZKX4HvrbnjB1hVN1n0hptJZJGDLTos-Y_zvm4JGKDJ3yjidyjcoLeee6-HcM9dHnPw3wS3cO-4efz9HMz5Bf5z5J-jn8d-n3aOOh468o-7lP8do7_dO77z3V_cTer51fPdlbKHgbrg3vv8y_rR4bHyHfvgdD43vke_Nb0FfUfY5-vPMvzhF3y7nF_vYwlEcMi-4ZQlu_cQEW5hiCzvYwi62cAP3LTzl8YAlScyzWx7nD3TCX7hVt_Aoms7S7HLfwoOYM9x6xOkonTDcwufB9YSdsSBk3K5jC4csDUaTOcz4rj3lo9uAP2AL96ZBlLRQzc6A_VnaQu2sjOsgHdywBMWzdJpdzL4inU0nS5ey2zo-Ske3rIXqSfbQQ8oSxFkQtlATfcMW5vH9ywWC-08WXjy8oPJS6vUDugmSm9Ui29n9fQsnaTBkuEWerO247-e4369zt0nV2dON7F-_J-Yh4yzMf8_nDFjqrjcaecL4kP2IRxHjNsmpaML-Tffa5POnr3w0vFm8fG2i1abyfWS_2GCWjuJI1M7V_r32ximijKPhkLNhkMbcJo21llvYX7TnWQZH3uWV559feRfd7l6bZPR6Fyd7bZq9-u5feOfPr_XxLKYTV00n4obRulrDehcnV52sZU727oxFIeNzqaA2tdtOSXJpSLRoFr3VpDf748W1eGrTnJYkiDRkiLh1AZP9FSZE3hIJPA5sWrOdSlsi2Zb9gcRAqDx7upF9iQOBfISBAChjeSDs7_pAKKwTYCCQ3RsIVN5IqISNOjXbrbSRkG3ZNyVstPLs6Ub2Jdoo_Qg2Cihj2UYPdt1GC-sEsFG6ezbqyBuJI2Gjbs1uVNpIyLbsDyVstPLs6Ub2Jdqo8xFsFFDGso02d91GC-sEsFFn92zUlTcSV8JGG7VK2wjZlnsDNtGaTVAQhYigOL1hvNJ9oBv7UKKduh_BTgGNLNvp4a7baWGdAHbq7p6dAv8cOGPJNI4Sltv6vv3N9ax_LByyRb-TeMYH7JTHgznM4q0_f26-dwlZki4-pYs3nejloyQNUtmtcf3thsx_9PIlNFVLWLbCgtjEUQHfxocLVWeyNVSpNSA4KSBLqkeW4hKaqiUo0M_JsiC4uiyB6ky2hiq1BgSnefD6Mvgqdj3_sCOs3FW1GtnZU6QoUtf6KwfAGwbBqUnmVC9zV6jiHHb-4YbwYbq_-hvQY837ekOLsHlibOWRrE5fLbQo8tecSYA_vOG4dqA3tAg7I8ZWHskyshSXoBZaFPlrziTAH95wXGsKHftQHFoO9YUWsK1ibLVssn1iAqrSO7rF4JpDCwCulzlZO4-KUkvpR6IiB9Z3b02Ro2rJ4OppROqsXuhIWjl1aN6lrJ1aS8wlALjmXYEYXD2SSGkTKEKpBe-gTUAdemMbWTsIr7g6ccXphKwdZUuMJwC45t0JhK48ELdLR0BZmhMKhK45ooh3I6YjismtCQBuej4b3ahA6KXslCB1aG6BydUJAG56Phtdq0DopSyWIHVoboF4f0KABQoxuUEBwHVHFKOrEgBdLYkoBCSjWxRa6S0K1bhF0V6d3jULAK45okDoRrmrZxOZIUQ1blHeQZsm1ywAuOaIAqEb5a6eTaS0Kd6iUGCLQk1uUQBwzREFQtc8jY0uSyB0QxskWq0tSv_pt_8DAAD__3l-CW8= +https://cockroachdb.github.io/distsqlplan/decode.html#eJzcWWFP4zgU_H6_wvInVptuYieFEmmlsnvcqauScBSk41YVCo2vVC1Jz0lhEeK_n9KCaFPqF6eOCf3WpnHmzctjxmMecfLfBLu4d9w9_n6OZnyC_jjzT9DP479Pu0cdDx15R93Lf47R3u-d3nnvr-4n9Hzr-O7L2EDB3XBvfP9l_GmxbHyHfvgdD43vke_Nb0FfUfY7-vPMvzhF3y7nF_vYwFEcMi-4ZQl2f2KCDUyxgW1sYAcbuIn7Bp7yeMCSJObZLY_zBZ3wF3YtA4-i6SzNLvcNPIg5w-4jTkfphGEXnwfXE3bGgpBx08IGDlkajCZzmPFde8pHtwF_wAbuTYMocVHDzID9WeqidlbGdZAObliC4lk6zS5mj0hn08nSpey2jo_S0S1zkZVkix5SliDOgtBFLfQNG5jH9y8XCO4_GXixeEHlpdTrB3QTJDerRbaz-_sGTtJgyLBLnoxy3Pdz3O_XuZuk7uzpRvavz4l5yDgL88_5nAEXuuuNRp4wPmQ_4lHEuElyUzRh_6Z7bfL501c-Gt4sPr420WjT4n1kv9hglo7iSNTO1f699saWmYyj4ZCzYZDG3CTNtZYb2F-053kMjrzLK88_v_Iuut29Nsno9S5O9to0-_Tdv_DOnz9Xx1NuTpzt5kTcMGpt17DexclVJ2uZnX07Y1HI-HxUUJuabVvTuDQLtGgWvdWkN_vjxY14atLcLBUg0ixCxLEETPZXmJDikkhgOzBpw7RrLYmkLPuDAoZQe_Z0I3uNhkA-giEAk7FsCPu7bgjScwIYAtk9Q6DFhYQWkFG7YTq1FhJSln2rgIzWnj3dyF6jjNKPIKPAZCzL6MGuy6j0nAAySndPRu3iQmIXkFGnYTZrLSSkLPvDAjJae_Z0I3uNMmp_BBkFJmNZRlu7LqPScwLIqL17MuoUFxKngIw2G7WWEVKWexMW0YZJUBCFiKA4vWG81n2gG_ugUU6djyCnwIwsy-nhrsup9JwAcursnpwC_xw4Y8k0jhKWO_V9-8lW1j8WDtmi30k84wN2yuPBHGbx1Z-vm5-7hCxJF7_SxZdO9PJTkgZp0VNj6-2GzP_oi5fQkiphWfckgYhdGqmIwkqVoo00LU96DYlIjBKtZpTEJbSkStiCa26UZJDkRwkoRRtpWp70GhLNI1nLSKtAVn6xLSzT2UbSZDRepihiqfszBJCaupCoNk5UISdHOHk5oPzipnAx3V-d22okcF-hoQs7JQaSc7BKuG5h6LJICv0aeH86NykHCg1dyFkMJOdgJUdJXMIWhi6LpNCvgfenc5PSEirjodjQDxUaurBhYiBJk1a2dQCqUmh-YiSVhg4gKeRE1gKQyNGr3anLJKTtSMsEoSqRJD24bOyTykDv-5JVBu61mFSVQQNIKjOmGEnSiMvOE1BEeb4l5gl4yQp3JmQtZq1IJXHENk3WslNVPg0gqUzTEJScfyjbEwBlqbRqCEqlV4sDtFav1pajASSt3qUvUENQWs4PoJeskq-2MA0gafUufakagtJyiAC9ZJV8xYmaAJGaaMvUAJJSr9aXlAEoSf9Vty3Ql6tpfXI1rSpXKy9FYfAGkFR6NQSlj5WkI5eUcVpVri4xT9qCN4Ck0qshKH2sJB257DyJczUFcjXVlqsBJJVeDUGpdCp9WRmCeqfTAvqOubr_9Nv_AQAA__-zE29D # This query verifies stats collection for the hashJoiner, distinct and sorter. query T SELECT url FROM [EXPLAIN ANALYZE (DISTSQL) SELECT DISTINCT(kw.w) FROM kv JOIN kw ON kv.k = kw.w ORDER BY kw.w] ---- -https://cockroachdb.github.io/distsqlplan/decode.html#eJzUWV1v4kYUfe-vGM1Topq1Z2xIgrRSsptUZZXCNslDtyseHDwNFsSm9hA2ivLfK0MRMWbnjplc27zhj_G5X5x77swLTf-d0i69vbq--nxH5smU_HYz-IN8v_rr6_VFr08u-hfX3_6-IkeXvdu72z-vj8n_r2bXvf7nO3I0WXxYHK-WTZ7Il0GvTyYLMuiTydOHCflIsudkcHN5dUM-fVteDalFozgQff9RpLT7nTJqUU4t6lKLetSibTq06CyJRyJN4yR75WW5oBf8oF3HomE0m8vs9tCiozgRtPtCZSingnbpnX8_FTfCD0RiO9SigZB-OF3CTJ7OZ0n46CfP1KK3Mz9Ku6RlZ8CDueyS88yMe1-OxiIl8VzOspvZJ-R8Nn1zK3utNyAyfBRd4qTZomcpUpIIP-iSU_KJWjSJF-sbjA5fLbpavHJlber9Mxn76Thv5Hn2_tCiqfQfBO2yV2s_3ztbvi-Kvtvsrfe8gd7zn3q_-c48ipNAJCLIfWmYrYRe2RHC3_10_CUOI5HYbKt8puIfeXTOjj8m4cN4-Wuf2IkfYjSXYRypQpiP2SYebplquAxTGUYjabN2IciY5pZLsWeQYpXT3KnU6Y1D7TI5uo0TKRKbFzL0a0XWdjTCvyv4SxN3ZqAft-KZ7W6FX8OXto4vnqNw5iTnDNMnSwY3Cpu3bPeAWkUJ7080WkXO--a3ClZxq2ANbxVANWxaRedgWkW5FCtbBWtEqwBytG4VhQzV0yq4Pr9wDXZ1W7Z3QOxawvtTDXbNed98duUVsytvOLsC1bBh15ODYddyKVayK28EuwI5WrNrIUP1sKurzy-uBrt6Lbt9QOxawvszDXbNed98dnUrZle34ewKVMOGXU8Phl3LpVjJrm4j2BXI0ZpdCxmqh109fX7xNNi13Togbi3hexvm1pbNiB8FhJFYjkVyQCzrVcyyXsNZFqiLDcueHQzLlkuxkmW9RrAskKM1yxYyVP9m8g5rb0Q6i6NUbP_Tdn7ZydIjggexymUaz5OR-JrEoyXM6nKwXLfcRwlEKldP-eqiF60fpdKXujvSzu6ALOlA34QzUxPeMmRJbNY2Ad-Hnkslh9UYGn6CCs5KlCXHKUu1CWemJhi4v1WWJcHNyxJIDqsxNFtl-d7gfBvceQvu5rCd7cWu0nLPlGp0e08ZoxguxQDguBSjBuduneC4Tc9TVnFbXcVt5eJO3vDtxR31_ydfbQ4Or5_gKh5l5NXYxlk3d99M8Rj6jyxoAHAjtjFvqqe4ikcZGTW2sdTRKUu1CWaKx9B_ZEEDgBsJGvOyPFMyNnPUzYKphwgjzQMrRWC4M2Kb_RUXZBbyYAWMdbiyB0LH1T2sMDvki5kDxaxW_516-wcDJCVuZNXgyGkFxghjSaO1WwCE35hscKsDmXQKE0eF4gYAR97IUYOb6xqt2gTCb6Rv3qE2AfNwt3SYeqBlHaApFEbaKhWOGhx59wZCNx6o9tRXarOwFQ6AjtwKCzNuvphPgWIuzCKNUjiFWaRKhaMGN0oremiQd3WgxBjTkFaTA4zAVTi8zDnqu59KlDlBrby9A9Yhb-1AiTHe29GpTcgI5EMr9djLgbGXF8beChUOAI6scCB0ZE5Ro5sJmf31FWQWrsLhhUE2X8weUMzqs6-aFQ4vzCLvp3DQrUPOuxocWeFAiUGmIQDdmIa0ehgw2NQsMtTW4W7yAODYCgdIDO7pFYRuLG20alM99nJg7OXqIzhkhQOcfCJTC4COrHCAs0_crRoI3Xim31NfqY9cXeDI1S3MGrUqnOHrL_8FAAD__0rC1W4= +https://cockroachdb.github.io/distsqlplan/decode.html#eJzUWV1vo0YUfe-vGM1TouKFGcBJkFZKdpOqXqX2NslDtys_EDONkR1wYZxsFOW_V9i1HIx3LgPXGL8ZzHDu15xzL_NK03-n1KO3V9dXn-_IPJmS324Gf5DvV399vb7o9clF_-L6299X5Oiyd3t3--f1Mfn_0ey61_98R44mzx-ej5fLJk_ky6DXJ5NnMuiTydOHCflIsv_J4Oby6oZ8-ra4GlKDRnEg-v6jSKn3nTJqUE4NalODOtSgLh0adJbEI5GmcZI98rpY0At-UM8yaBjN5jK7PTToKE4E9V6pDOVUUI_e-fdTcSP8QCSmRQ0aCOmH0wXM5Ol8loSPfvJCDXo786PUIx0zAx7MpUfOMzPufTkai5TEcznLbmavkPPZ9N2t7LHegMjwUXjESrNFL1KkJBF-4JFT8okaNImfVzcYHb4ZdLl46crK1PsXMvbTcd7I8-z5oUFT6T8I6rE3o5rv3Q3fn4u-m-y997yF3vOfer9-zzyKk0AkIsi9aZithB7ZEsLf_XT8JQ4jkZhso3ym4h95dM6OPybhw3jxq0rsxA8xmsswjlQhzMdsHQ9bpxouw1SG0UiazC0EeZfm6qXYqZFildPcatTptUOuTo5u40SKxOSFDP3akLXdEuHfFvyFiVsz0I878cy0N8Jfwhe3jC-OpXDmJOcMK0-WDBYKk3dM-4CkQsP7kxJSkfO-_VLBGpYK1nKpAKphLRXdg5EKvRQrpYK1QiqAHK2kopCh_UgFL88vvAS72h3TOSB21fD-tAS75rxvP7vyhtmVt5xdgWpYs-vJwbCrXoqV7Mpbwa5AjlbsWsjQftjVLs8vdgl2dTqme0DsquH9WQl2zXnffna1G2ZXu-XsClTDml1PD4Zd9VKsZFe7FewK5GjFroUM7YddnfL84pRgV7dzQNyq4bsLc2vHZMSPAsJILMciOSCWdRpmWaflLAvUxZplzw6GZfVSrGRZpxUsC-RoxbKFDO3_Y_IWa29EOoujVGzutK1vtrL0iOBBLHOZxvNkJL4m8WgBs7wcLNYtvqMEIpXLf_nyohet_kqlL8t-kba2B2RBB-VNONMy4T0dagIxtzJSGeLVCjtryml-gofENEqJ76aU1CacaZlQw9eNUtJB0i8lIOysKac3SqkWEt9Est4j2Tkga3OxrTTTqUNpOhyvYxRD3PAAEuKGVyNxuzEkRI1wlJXnqivPVS7u5q3cXNxV13y-Qqzd8OcJYjegDLMaSC-fO_G1RjegG1XEvQ8gVd_7-rJ0itgNKH1WA-lpfsVSUptQoxvQjSqi2ANI1cVev5TOlMzILDUpM3UHrNcPqPsjYMDQ2-horQdkFuYIAEwbiC0BBIXYE7BC45svQA4UoLqb7TZIygxopBBjpkbCzA7Q8OoJedXBEwhsdY3HTzLmfi-0y7tSeQAJc7RXI2mqedV6AgJbXegr1BNgC-Lcz9QTFOsCTFuYoXYm9WokzHkegtLbznhdhdosVKkHoDDFpDBB5QvwFCjAQte8P6kvdM07k3o1UvXsoDuNOdRD8a3V7JeWCcAIRKnnOidF9T7-6hwI7Vb6AFMwJ3sovnqKX7GeICMwP_GrhyoODFW8MFTtSuoBJEyph6Awt7MaSlO-0boKyCxEqeeFMSlfgA5QgOqTgialnhe6ZiSpRzcFM31qJEyph-KLyQAAVK1mv7QKAP12kwKsNgVx7AeQUKUeiC_iV3wISk_Qq9aTeqjiwFDF1UcRmFIPHORgbnQAClPqgaMcxEkdgqp1sFujq1CfINnACZJd6Iqbk_rh2y__BQAA__8jYTtR # This query verifies stats collection for WITH ORDINALITY and the hashJoiner. query T @@ -91,7 +91,7 @@ https://cockroachdb.github.io/distsqlplan/decode.html#eJyUkF9LwzAUxd_9FOE-dRBY42 query T SELECT url FROM [EXPLAIN ANALYZE (DISTSQL) SELECT avg(k) OVER () FROM kv] ---- -https://cockroachdb.github.io/distsqlplan/decode.html#eJzMlE2P2jwQx-_Pp7DmBJJRXnlU-QS0tEKlsAW0fVnl4I1HNCKJU9thQYjvXjnZtmS1dNNDpRwznv_M_zcTzQn09xQYrKfz6esNKVVK3q6WH8jd9PPNfDxbkPFiPP_ydUp6b2brzfrjvE8eU_l-29v1yfJ2uiK9fq3a7SOgkEuBC56hBnYHHlDwgUIAFEKgMISIQqFkjFpLZVNOlWAmDsBcCklelMaGIwqxVAjsBCYxKQKDDb9PcYVcoHJcoCDQ8CSt2uz2o0IlGVdHoLAueK4ZGTi28bI0jIysjXtu4m-oiSxNYYO2hCmL9CJk02ZLYpIMGXG1FR0NaqKQC0ZekQlQUPLhZ8CD6EyhFj961oZvEZh3ple4fuOUuVQCFYoGSnR-hvxTkgv5gMoZNrHHt-96I6__C9Nv-rPZ2vA0vSTK-IFkmEl1JKVGwUjokvfJpN2AbEU8YFyaRObX5hS6ZHJ1Mn5jMl77jXsvb9zxB07QkZ377cn8FmTBwAk7Qha0JwtakIWD6q_uAlnYnixsQTYcdITrhbu6Ql3IXOOTO_R8ZdfeJxRbrI-ZlqWK8UbJuGpTfy4rXRUQqE396tUfs7x6qgxeir0_iv9viN2nYv9vOlcoFdW_WknTXNBlc2GXzQ07ZS46__cjAAD__02r6hs= +https://cockroachdb.github.io/distsqlplan/decode.html#eJzMlE1v2kAQhu_9Fas5gbTIn1TVnkJaWqFSSAGlH5EPG--IWthed3dNQIj_Xq2dtjiCxjnVR8_OO_M-M9YcQP9MgcFyPB2_XZFSpeT9Yv6J3I2_3kxHkxkZzUbTb9_HpPduslwtP0_75DGVb9e9TZ_Mb8cL0uvXqs02Agq5FDjjGWpgd-ABBR8oBEAhBApDiCgUSsaotVQ25VAJJmIHzKWQ5EVpbDiiEEuFwA5gEpMiMFjx-xQXyAUqxwUKAg1P0qrNZntVqCTjag8UlgXPNSMDxzael4aRK2vjnpv4B2oiS1PYoC1hyiI9Cdm0yZyYJENGXG1Fe4OaKOSCkTfkGigo-fA74EF0pFCLHz1rw9cIzDvSC1x_ccpcKoEKRQMlOp4h_5LkQj6gcoZN7NHth96V1_-D6Tf92WxteJqeEmV8RzLMpNqTUqNgJHTJx-S63YBsRdxhXJpE5pfmFLrk-uJk_MZkvPYb957fuOMPnKAjO_fbk_ktyIKBE3aELGhPFrQgCwfVX90FsrA9WdiCbDjoCNczd3WBupC5xid36Hxl194nFGusj5mWpYrxRsm4alN_zitdFRCoTf3q1R-TvHqqDJ6KvX-KXzfE7lOx_5LOFUpFdXYlL51_00nQGSdhZ5wM_5-T6PjqVwAAAP__3R_Qdw== # Very simple query to make it easier to spot regressions when rewriting results # in test files. diff --git a/pkg/sql/physicalplan/physical_plan.go b/pkg/sql/physicalplan/physical_plan.go index dc318a6d2b39..812e75be6610 100644 --- a/pkg/sql/physicalplan/physical_plan.go +++ b/pkg/sql/physicalplan/physical_plan.go @@ -931,13 +931,14 @@ func (p *PhysicalPlan) PopulateEndpoints() { } else { endpoint.Type = execinfrapb.StreamEndpointSpec_REMOTE } - p2.Spec.Input[s.DestInput].Streams = append(p2.Spec.Input[s.DestInput].Streams, endpoint) if endpoint.Type == execinfrapb.StreamEndpointSpec_REMOTE { if !p.remotePlan { p.remotePlan = true } + endpoint.OriginNodeID = p1.Node endpoint.TargetNodeID = p2.Node } + p2.Spec.Input[s.DestInput].Streams = append(p2.Spec.Input[s.DestInput].Streams, endpoint) router := &p1.Spec.Output[0] // We are about to put this stream on the len(router.Streams) position in