distsql

package
v0.0.0-...-5697661 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Jul 17, 2016 License: Apache-2.0 Imports: 18 Imported by: 0

Documentation

Overview

Package distsql is a generated protocol buffer package.

It is generated from these files:

cockroach/sql/distsql/api.proto
cockroach/sql/distsql/data.proto
cockroach/sql/distsql/processors.proto

It has these top-level messages:

SetupFlowsRequest
SimpleResponse
Expression
Ordering
MailboxSpec
StreamEndpointSpec
InputSyncSpec
OutputRouterSpec
DatumInfo
StreamHeader
StreamData
StreamTrailer
StreamMessage
TableReaderSpan
TableReaderSpec
JoinReaderSpec
ProcessorCoreUnion
ProcessorSpec
FlowSpec

Index

Constants

This section is empty.

Variables

View Source
var (
	ErrInvalidLengthApi = fmt.Errorf("proto: negative length found during unmarshaling")
	ErrIntOverflowApi   = fmt.Errorf("proto: integer overflow")
)
View Source
var (
	ErrInvalidLengthData = fmt.Errorf("proto: negative length found during unmarshaling")
	ErrIntOverflowData   = fmt.Errorf("proto: integer overflow")
)
View Source
var (
	ErrInvalidLengthProcessors = fmt.Errorf("proto: negative length found during unmarshaling")
	ErrIntOverflowProcessors   = fmt.Errorf("proto: integer overflow")
)
View Source
var InputSyncSpec_Type_name = map[int32]string{
	0: "UNORDERED",
	1: "ORDERED",
}
View Source
var InputSyncSpec_Type_value = map[string]int32{
	"UNORDERED": 0,
	"ORDERED":   1,
}
View Source
var Ordering_Column_Direction_name = map[int32]string{
	0: "ASC",
	1: "DESC",
}
View Source
var Ordering_Column_Direction_value = map[string]int32{
	"ASC":  0,
	"DESC": 1,
}
View Source
var OutputRouterSpec_Type_name = map[int32]string{
	0: "MIRROR",
	1: "BY_HASH",
	2: "BY_RANGE",
}
View Source
var OutputRouterSpec_Type_value = map[string]int32{
	"MIRROR":   0,
	"BY_HASH":  1,
	"BY_RANGE": 2,
}

Functions

func RegisterDistSQLServer

func RegisterDistSQLServer(s *grpc.Server, srv DistSQLServer)

Types

type DatumInfo

type DatumInfo struct {
	Encoding cockroach_sql_sqlbase2.DatumEncoding   `protobuf:"varint,1,opt,name=encoding,enum=cockroach.sql.sqlbase.DatumEncoding" json:"encoding"`
	Type     cockroach_sql_sqlbase1.ColumnType_Kind `protobuf:"varint,2,opt,name=type,enum=cockroach.sql.sqlbase.ColumnType_Kind" json:"type"`
}

func (*DatumInfo) Descriptor

func (*DatumInfo) Descriptor() ([]byte, []int)

func (*DatumInfo) Marshal

func (m *DatumInfo) Marshal() (data []byte, err error)

func (*DatumInfo) MarshalTo

func (m *DatumInfo) MarshalTo(data []byte) (int, error)

func (*DatumInfo) ProtoMessage

func (*DatumInfo) ProtoMessage()

func (*DatumInfo) Reset

func (m *DatumInfo) Reset()

func (*DatumInfo) Size

func (m *DatumInfo) Size() (n int)

func (*DatumInfo) String

func (m *DatumInfo) String() string

func (*DatumInfo) Unmarshal

func (m *DatumInfo) Unmarshal(data []byte) error

type DistSQLClient

type DistSQLClient interface {
	// RunSimpleFlow instantiates a flow and streams back results of that flow.
	// The request must contain one flow, and that flow must have a single mailbox
	// of the special simple response type.
	RunSimpleFlow(ctx context.Context, in *SetupFlowsRequest, opts ...grpc.CallOption) (DistSQL_RunSimpleFlowClient, error)
	// SetupFlows instantiates a set of flows (subgraphs of a distributed SQL
	// computation) on the receiving node.
	SetupFlows(ctx context.Context, in *SetupFlowsRequest, opts ...grpc.CallOption) (*SimpleResponse, error)
	// FlowStream is used to push a stream of messages that is part of a flow. The
	// first message will have a StreamHeader which identifies the flow and the
	// stream (mailbox).
	FlowStream(ctx context.Context, opts ...grpc.CallOption) (DistSQL_FlowStreamClient, error)
}

func NewDistSQLClient

func NewDistSQLClient(cc *grpc.ClientConn) DistSQLClient

type DistSQLServer

type DistSQLServer interface {
	// RunSimpleFlow instantiates a flow and streams back results of that flow.
	// The request must contain one flow, and that flow must have a single mailbox
	// of the special simple response type.
	RunSimpleFlow(*SetupFlowsRequest, DistSQL_RunSimpleFlowServer) error
	// SetupFlows instantiates a set of flows (subgraphs of a distributed SQL
	// computation) on the receiving node.
	SetupFlows(context.Context, *SetupFlowsRequest) (*SimpleResponse, error)
	// FlowStream is used to push a stream of messages that is part of a flow. The
	// first message will have a StreamHeader which identifies the flow and the
	// stream (mailbox).
	FlowStream(DistSQL_FlowStreamServer) error
}

type DistSQL_FlowStreamClient

type DistSQL_FlowStreamClient interface {
	Send(*StreamMessage) error
	CloseAndRecv() (*SimpleResponse, error)
	grpc.ClientStream
}

type DistSQL_FlowStreamServer

type DistSQL_FlowStreamServer interface {
	SendAndClose(*SimpleResponse) error
	Recv() (*StreamMessage, error)
	grpc.ServerStream
}

type DistSQL_RunSimpleFlowClient

type DistSQL_RunSimpleFlowClient interface {
	Recv() (*StreamMessage, error)
	grpc.ClientStream
}

type DistSQL_RunSimpleFlowServer

type DistSQL_RunSimpleFlowServer interface {
	Send(*StreamMessage) error
	grpc.ServerStream
}

type Expression

type Expression struct {
	// TODO(radu): TBD how this will be used
	Version string `protobuf:"bytes,1,opt,name=version" json:"version"`
	// SQL expressions are passed as a string, with Placeholders ($1, $2 ..) used for
	// "input" variables.
	Expr string `protobuf:"bytes,2,opt,name=expr" json:"expr"`
}

func (*Expression) Descriptor

func (*Expression) Descriptor() ([]byte, []int)

func (*Expression) Marshal

func (m *Expression) Marshal() (data []byte, err error)

func (*Expression) MarshalTo

func (m *Expression) MarshalTo(data []byte) (int, error)

func (*Expression) ProtoMessage

func (*Expression) ProtoMessage()

func (*Expression) Reset

func (m *Expression) Reset()

func (*Expression) Size

func (m *Expression) Size() (n int)

func (*Expression) String

func (m *Expression) String() string

func (*Expression) Unmarshal

func (m *Expression) Unmarshal(data []byte) error

type Flow

type Flow struct {
	// contains filtered or unexported fields
}

Flow represents a flow which consists of processors and streams.

func (*Flow) RunSync

func (f *Flow) RunSync()

RunSync runs the processors in the flow in order (serially), in the same context (no goroutines are spawned).

func (*Flow) Start

func (f *Flow) Start()

Start starts the flow (each processor runs in their own goroutine).

func (*Flow) Wait

func (f *Flow) Wait()

Wait waits for all the goroutines for this flow to exit.

type FlowID

type FlowID struct {
	uuid.UUID
}

FlowID identifies a flow. It is most importantly used when setting up streams between nodes.

type FlowSpec

type FlowSpec struct {
	FlowID     FlowID          `protobuf:"bytes,1,opt,name=flow_id,json=flowId,customtype=FlowID" json:"flow_id"`
	Processors []ProcessorSpec `protobuf:"bytes,2,rep,name=processors" json:"processors"`
}

FlowSpec describes a "flow" which is a subgraph of a distributed SQL computation consisting of processors and streams.

func (*FlowSpec) Descriptor

func (*FlowSpec) Descriptor() ([]byte, []int)

func (*FlowSpec) Marshal

func (m *FlowSpec) Marshal() (data []byte, err error)

func (*FlowSpec) MarshalTo

func (m *FlowSpec) MarshalTo(data []byte) (int, error)

func (*FlowSpec) ProtoMessage

func (*FlowSpec) ProtoMessage()

func (*FlowSpec) Reset

func (m *FlowSpec) Reset()

func (*FlowSpec) Size

func (m *FlowSpec) Size() (n int)

func (*FlowSpec) String

func (m *FlowSpec) String() string

func (*FlowSpec) Unmarshal

func (m *FlowSpec) Unmarshal(data []byte) error

type InputSyncSpec

type InputSyncSpec struct {
	Ordering Ordering             `protobuf:"bytes,1,opt,name=ordering" json:"ordering"`
	Streams  []StreamEndpointSpec `protobuf:"bytes,2,rep,name=streams" json:"streams"`
}

InputSyncSpec is the specification for an input synchronizer; it decides how to interleave rows from multiple input streams.

func (*InputSyncSpec) Descriptor

func (*InputSyncSpec) Descriptor() ([]byte, []int)

func (*InputSyncSpec) Marshal

func (m *InputSyncSpec) Marshal() (data []byte, err error)

func (*InputSyncSpec) MarshalTo

func (m *InputSyncSpec) MarshalTo(data []byte) (int, error)

func (*InputSyncSpec) ProtoMessage

func (*InputSyncSpec) ProtoMessage()

func (*InputSyncSpec) Reset

func (m *InputSyncSpec) Reset()

func (*InputSyncSpec) Size

func (m *InputSyncSpec) Size() (n int)

func (*InputSyncSpec) String

func (m *InputSyncSpec) String() string

func (*InputSyncSpec) Unmarshal

func (m *InputSyncSpec) Unmarshal(data []byte) error

type InputSyncSpec_Type

type InputSyncSpec_Type int32
const (
	// Rows from the input streams are interleaved arbitrarily.
	InputSyncSpec_UNORDERED InputSyncSpec_Type = 0
	// The input streams are guaranteed to be ordered according to the column
	// ordering field; rows from the streams are interleaved to preserve that
	// ordering.
	InputSyncSpec_ORDERED InputSyncSpec_Type = 1
)

func (InputSyncSpec_Type) Enum

func (InputSyncSpec_Type) EnumDescriptor

func (InputSyncSpec_Type) EnumDescriptor() ([]byte, []int)

func (InputSyncSpec_Type) String

func (x InputSyncSpec_Type) String() string

func (*InputSyncSpec_Type) UnmarshalJSON

func (x *InputSyncSpec_Type) UnmarshalJSON(data []byte) error

type JoinReaderSpec

type JoinReaderSpec struct {
	Table cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"`
	// If 0, we use the primary index; each row in the input stream has a value
	// for each primary key.
	// TODO(radu): figure out the correct semantics when joining with an index.
	IndexIdx uint32 `protobuf:"varint,2,opt,name=index_idx,json=indexIdx" json:"index_idx"`
	// The filter expression references the columns in the table (table.columns)
	// via $0, $1, etc. If a secondary index is used, the columns that are not
	// available as part of the index cannot be referenced.
	Filter Expression `protobuf:"bytes,3,opt,name=filter" json:"filter"`
	// The table reader will only produce values for these columns, referenced by
	// their indices in table.columns.
	OutputColumns []uint32 `protobuf:"varint,4,rep,packed,name=output_columns,json=outputColumns" json:"output_columns,omitempty"`
}

JoinReaderSpec is the specification for a "join reader". A join reader performs KV operations to retrieve specific rows that correspond to the values in the input stream (join by lookup).

func (*JoinReaderSpec) Descriptor

func (*JoinReaderSpec) Descriptor() ([]byte, []int)

func (*JoinReaderSpec) Marshal

func (m *JoinReaderSpec) Marshal() (data []byte, err error)

func (*JoinReaderSpec) MarshalTo

func (m *JoinReaderSpec) MarshalTo(data []byte) (int, error)

func (*JoinReaderSpec) ProtoMessage

func (*JoinReaderSpec) ProtoMessage()

func (*JoinReaderSpec) Reset

func (m *JoinReaderSpec) Reset()

func (*JoinReaderSpec) Size

func (m *JoinReaderSpec) Size() (n int)

func (*JoinReaderSpec) String

func (m *JoinReaderSpec) String() string

func (*JoinReaderSpec) Unmarshal

func (m *JoinReaderSpec) Unmarshal(data []byte) error

type LocalStreamID

type LocalStreamID int

LocalStreamID identifies a stream that is local to a flow. The identifier can only be used in the context of a specific flow.

type MailboxSpec

type MailboxSpec struct {
	// If set, we are in a special "simple flow" mode. In this mode, we return
	// results directly as part of the RPC call that set up the flow. This saves
	// overhead (extra RPCs) compared to the normal mode where the RPC just sets
	// up the flow. In this mode there are no remote endpoints so this must be the
	// only mailbox in that flow.
	SimpleResponse bool `protobuf:"varint,1,opt,name=simple_response,json=simpleResponse" json:"simple_response"`
	// The ID used to refer to this mailbox in a StreamHeader.
	StreamID StreamID `protobuf:"varint,2,opt,name=stream_id,json=streamId,casttype=StreamID" json:"stream_id"`
	// Serving address for the target host, used for outgoing mailboxes.
	TargetAddr string `protobuf:"bytes,3,opt,name=target_addr,json=targetAddr" json:"target_addr"`
}

A mailbox is where a stream "crosses" hosts - it is an endpoint of a local stream which receives data from or sends to a remote host.

func (*MailboxSpec) Descriptor

func (*MailboxSpec) Descriptor() ([]byte, []int)

func (*MailboxSpec) Marshal

func (m *MailboxSpec) Marshal() (data []byte, err error)

func (*MailboxSpec) MarshalTo

func (m *MailboxSpec) MarshalTo(data []byte) (int, error)

func (*MailboxSpec) ProtoMessage

func (*MailboxSpec) ProtoMessage()

func (*MailboxSpec) Reset

func (m *MailboxSpec) Reset()

func (*MailboxSpec) Size

func (m *MailboxSpec) Size() (n int)

func (*MailboxSpec) String

func (m *MailboxSpec) String() string

func (*MailboxSpec) Unmarshal

func (m *MailboxSpec) Unmarshal(data []byte) error

type Ordering

type Ordering struct {
	Columns []Ordering_Column `protobuf:"bytes,1,rep,name=columns" json:"columns"`
}

Ordering defines an order - specifically a list of column indices and directions. See sqlbase.ColumnOrdering.

func (*Ordering) Descriptor

func (*Ordering) Descriptor() ([]byte, []int)

func (*Ordering) Marshal

func (m *Ordering) Marshal() (data []byte, err error)

func (*Ordering) MarshalTo

func (m *Ordering) MarshalTo(data []byte) (int, error)

func (*Ordering) ProtoMessage

func (*Ordering) ProtoMessage()

func (*Ordering) Reset

func (m *Ordering) Reset()

func (*Ordering) Size

func (m *Ordering) Size() (n int)

func (*Ordering) String

func (m *Ordering) String() string

func (*Ordering) Unmarshal

func (m *Ordering) Unmarshal(data []byte) error

type Ordering_Column

type Ordering_Column struct {
	ColIdx    uint32                    `protobuf:"varint,1,opt,name=col_idx,json=colIdx" json:"col_idx"`
	Direction Ordering_Column_Direction `protobuf:"varint,2,opt,name=direction,enum=cockroach.sql.distsql.Ordering_Column_Direction" json:"direction"`
}

func (*Ordering_Column) Descriptor

func (*Ordering_Column) Descriptor() ([]byte, []int)

func (*Ordering_Column) Marshal

func (m *Ordering_Column) Marshal() (data []byte, err error)

func (*Ordering_Column) MarshalTo

func (m *Ordering_Column) MarshalTo(data []byte) (int, error)

func (*Ordering_Column) ProtoMessage

func (*Ordering_Column) ProtoMessage()

func (*Ordering_Column) Reset

func (m *Ordering_Column) Reset()

func (*Ordering_Column) Size

func (m *Ordering_Column) Size() (n int)

func (*Ordering_Column) String

func (m *Ordering_Column) String() string

func (*Ordering_Column) Unmarshal

func (m *Ordering_Column) Unmarshal(data []byte) error

type Ordering_Column_Direction

type Ordering_Column_Direction int32

The direction of the desired ordering for a column.

const (
	Ordering_Column_ASC  Ordering_Column_Direction = 0
	Ordering_Column_DESC Ordering_Column_Direction = 1
)

func (Ordering_Column_Direction) Enum

func (Ordering_Column_Direction) EnumDescriptor

func (Ordering_Column_Direction) EnumDescriptor() ([]byte, []int)

func (Ordering_Column_Direction) String

func (x Ordering_Column_Direction) String() string

func (*Ordering_Column_Direction) UnmarshalJSON

func (x *Ordering_Column_Direction) UnmarshalJSON(data []byte) error

type OutputRouterSpec

type OutputRouterSpec struct {
	Type    OutputRouterSpec_Type `protobuf:"varint,1,opt,name=type,enum=cockroach.sql.distsql.OutputRouterSpec_Type" json:"type"`
	Streams []StreamEndpointSpec  `protobuf:"bytes,2,rep,name=streams" json:"streams"`
}

OutputRouterSpec is the specification for the output router of a processor; it decides how to send results to multiple output streams.

func (*OutputRouterSpec) Descriptor

func (*OutputRouterSpec) Descriptor() ([]byte, []int)

func (*OutputRouterSpec) Marshal

func (m *OutputRouterSpec) Marshal() (data []byte, err error)

func (*OutputRouterSpec) MarshalTo

func (m *OutputRouterSpec) MarshalTo(data []byte) (int, error)

func (*OutputRouterSpec) ProtoMessage

func (*OutputRouterSpec) ProtoMessage()

func (*OutputRouterSpec) Reset

func (m *OutputRouterSpec) Reset()

func (*OutputRouterSpec) Size

func (m *OutputRouterSpec) Size() (n int)

func (*OutputRouterSpec) String

func (m *OutputRouterSpec) String() string

func (*OutputRouterSpec) Unmarshal

func (m *OutputRouterSpec) Unmarshal(data []byte) error

type OutputRouterSpec_Type

type OutputRouterSpec_Type int32
const (
	// Each row is sent to all output streams (also used if there is a
	// single output stream).
	OutputRouterSpec_MIRROR OutputRouterSpec_Type = 0
	// Each row is sent to one stream, chosen by hashing certain columns of
	// the row. TODO(radu): an extra optional structure below for the hashing
	// details.
	OutputRouterSpec_BY_HASH OutputRouterSpec_Type = 1
	// Each row is sent to one stream, chosen according to preset boundaries
	// for the values of certain columns of the row. TODO(radu): an extra
	// optional structure below for the range details.
	OutputRouterSpec_BY_RANGE OutputRouterSpec_Type = 2
)

func (OutputRouterSpec_Type) Enum

func (OutputRouterSpec_Type) EnumDescriptor

func (OutputRouterSpec_Type) EnumDescriptor() ([]byte, []int)

func (OutputRouterSpec_Type) String

func (x OutputRouterSpec_Type) String() string

func (*OutputRouterSpec_Type) UnmarshalJSON

func (x *OutputRouterSpec_Type) UnmarshalJSON(data []byte) error

type ProcessorCoreUnion

type ProcessorCoreUnion struct {
	TableReader *TableReaderSpec `protobuf:"bytes,1,opt,name=tableReader" json:"tableReader,omitempty"`
}

func (*ProcessorCoreUnion) Descriptor

func (*ProcessorCoreUnion) Descriptor() ([]byte, []int)

func (*ProcessorCoreUnion) GetValue

func (this *ProcessorCoreUnion) GetValue() interface{}

func (*ProcessorCoreUnion) Marshal

func (m *ProcessorCoreUnion) Marshal() (data []byte, err error)

func (*ProcessorCoreUnion) MarshalTo

func (m *ProcessorCoreUnion) MarshalTo(data []byte) (int, error)

func (*ProcessorCoreUnion) ProtoMessage

func (*ProcessorCoreUnion) ProtoMessage()

func (*ProcessorCoreUnion) Reset

func (m *ProcessorCoreUnion) Reset()

func (*ProcessorCoreUnion) SetValue

func (this *ProcessorCoreUnion) SetValue(value interface{}) bool

func (*ProcessorCoreUnion) Size

func (m *ProcessorCoreUnion) Size() (n int)

func (*ProcessorCoreUnion) String

func (m *ProcessorCoreUnion) String() string

func (*ProcessorCoreUnion) Unmarshal

func (m *ProcessorCoreUnion) Unmarshal(data []byte) error

type ProcessorSpec

type ProcessorSpec struct {
	// In most cases, there is one input.
	Input []InputSyncSpec    `protobuf:"bytes,1,rep,name=input" json:"input"`
	Core  ProcessorCoreUnion `protobuf:"bytes,2,opt,name=core" json:"core"`
	// In most cases, there is one output.
	Output []OutputRouterSpec `protobuf:"bytes,3,rep,name=output" json:"output"`
}

func (*ProcessorSpec) Descriptor

func (*ProcessorSpec) Descriptor() ([]byte, []int)

func (*ProcessorSpec) Marshal

func (m *ProcessorSpec) Marshal() (data []byte, err error)

func (*ProcessorSpec) MarshalTo

func (m *ProcessorSpec) MarshalTo(data []byte) (int, error)

func (*ProcessorSpec) ProtoMessage

func (*ProcessorSpec) ProtoMessage()

func (*ProcessorSpec) Reset

func (m *ProcessorSpec) Reset()

func (*ProcessorSpec) Size

func (m *ProcessorSpec) Size() (n int)

func (*ProcessorSpec) String

func (m *ProcessorSpec) String() string

func (*ProcessorSpec) Unmarshal

func (m *ProcessorSpec) Unmarshal(data []byte) error

type RowBuffer

type RowBuffer struct {
	// contains filtered or unexported fields
}

RowBuffer is an implementation of RowReceiver that buffers (accumulates) results in memory, as well as an implementation of rowSender that returns rows from a row buffer.

func (*RowBuffer) Close

func (rb *RowBuffer) Close(err error)

Close is part of the RowReceiver interface.

func (*RowBuffer) NextRow

func (rb *RowBuffer) NextRow() (sqlbase.EncDatumRow, error)

NextRow is part of the RowSource interface.

func (*RowBuffer) PushRow

func (rb *RowBuffer) PushRow(row sqlbase.EncDatumRow) bool

PushRow is part of the RowReceiver interface.

type RowChannel

type RowChannel struct {
	// The channel on which rows are delivered.
	C <-chan StreamMsg
	// contains filtered or unexported fields
}

RowChannel is a thin layer over a StreamMsg channel, which can be used to transfer rows between goroutines.

func (*RowChannel) Close

func (rc *RowChannel) Close(err error)

Close is part of the RowReceiver interface.

func (*RowChannel) Init

func (rc *RowChannel) Init()

Init initializes the RowChannel with the default buffer size.

func (*RowChannel) InitWithBufSize

func (rc *RowChannel) InitWithBufSize(chanBufSize int)

InitWithBufSize initializes the RowChannel with a given buffer size.

func (*RowChannel) NextRow

func (rc *RowChannel) NextRow() (sqlbase.EncDatumRow, error)

NextRow is part of the RowSource interface.

func (*RowChannel) NoMoreRows

func (rc *RowChannel) NoMoreRows()

NoMoreRows causes future PushRow calls to return false. The caller should still drain the channel to make sure the sender is not blocked.

func (*RowChannel) PushRow

func (rc *RowChannel) PushRow(row sqlbase.EncDatumRow) bool

PushRow is part of the RowReceiver interface.

type RowReceiver

type RowReceiver interface {
	// PushRow sends a row to this receiver. May block.
	// Returns true if the row was sent, or false if the receiver does not need
	// any more rows. In all cases, Close() still needs to be called.
	// The sender must not use the row anymore after calling this function.
	PushRow(row sqlbase.EncDatumRow) bool
	// Close is called when we have no more rows; it causes the RowReceiver to
	// process all rows and clean up. If err is not null, the error is sent to
	// the receiver (and the function may block).
	Close(err error)
}

RowReceiver is any component of a flow that receives rows from another component. It can be an input synchronizer, a router, or a mailbox.

type RowSource

type RowSource interface {
	// NextRow retrieves the next row. Returns a nil row if there are no more
	// rows. Depending on the implementation, it may block.
	NextRow() (sqlbase.EncDatumRow, error)
}

RowSource is any component of a flow that produces rows that cam be consumed by another component.

type ServerContext

type ServerContext struct {
	DB *client.DB
}

ServerContext encompasses the configuration required to create a DistSQLServer.

type ServerImpl

type ServerImpl struct {
	// contains filtered or unexported fields
}

ServerImpl implements the server for the distributed SQL APIs.

func NewServer

func NewServer(ctx ServerContext) *ServerImpl

NewServer instantiates a DistSQLServer.

func (*ServerImpl) FlowStream

func (ds *ServerImpl) FlowStream(stream DistSQL_FlowStreamServer) error

FlowStream is part of the DistSQLServer interface.

func (*ServerImpl) RunSimpleFlow

func (ds *ServerImpl) RunSimpleFlow(
	req *SetupFlowsRequest, stream DistSQL_RunSimpleFlowServer,
) error

RunSimpleFlow is part of the DistSQLServer interface.

func (*ServerImpl) SetupFlows

func (ds *ServerImpl) SetupFlows(ctx context.Context, req *SetupFlowsRequest) (
	*SimpleResponse, error,
)

SetupFlows is part of the DistSQLServer interface.

func (*ServerImpl) SetupSimpleFlow

func (ds *ServerImpl) SetupSimpleFlow(
	ctx context.Context, req *SetupFlowsRequest, output RowReceiver,
) (*Flow, error)

SetupSimpleFlow sets up a simple flow, connecting the simple response output stream to the given RowReceiver. The flow is not started.

type SetupFlowsRequest

type SetupFlowsRequest struct {
	Txn   cockroach_roachpb1.Transaction `protobuf:"bytes,1,opt,name=txn" json:"txn"`
	Flows []FlowSpec                     `protobuf:"bytes,2,rep,name=flows" json:"flows"`
}

func (*SetupFlowsRequest) Descriptor

func (*SetupFlowsRequest) Descriptor() ([]byte, []int)

func (*SetupFlowsRequest) Marshal

func (m *SetupFlowsRequest) Marshal() (data []byte, err error)

func (*SetupFlowsRequest) MarshalTo

func (m *SetupFlowsRequest) MarshalTo(data []byte) (int, error)

func (*SetupFlowsRequest) ProtoMessage

func (*SetupFlowsRequest) ProtoMessage()

func (*SetupFlowsRequest) Reset

func (m *SetupFlowsRequest) Reset()

func (*SetupFlowsRequest) Size

func (m *SetupFlowsRequest) Size() (n int)

func (*SetupFlowsRequest) String

func (m *SetupFlowsRequest) String() string

func (*SetupFlowsRequest) Unmarshal

func (m *SetupFlowsRequest) Unmarshal(data []byte) error

type SimpleResponse

type SimpleResponse struct {
	Error *cockroach_roachpb2.Error `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"`
}

func (*SimpleResponse) Descriptor

func (*SimpleResponse) Descriptor() ([]byte, []int)

func (*SimpleResponse) Marshal

func (m *SimpleResponse) Marshal() (data []byte, err error)

func (*SimpleResponse) MarshalTo

func (m *SimpleResponse) MarshalTo(data []byte) (int, error)

func (*SimpleResponse) ProtoMessage

func (*SimpleResponse) ProtoMessage()

func (*SimpleResponse) Reset

func (m *SimpleResponse) Reset()

func (*SimpleResponse) Size

func (m *SimpleResponse) Size() (n int)

func (*SimpleResponse) String

func (m *SimpleResponse) String() string

func (*SimpleResponse) Unmarshal

func (m *SimpleResponse) Unmarshal(data []byte) error

type StreamData

type StreamData struct {
	// Encodes one or more data rows. Each datum is encoded according to the
	// corresponding DatumInfo.
	RawBytes []byte `protobuf:"bytes,1,opt,name=raw_bytes,json=rawBytes" json:"raw_bytes,omitempty"`
}

StreamData is a message that can be sent multiple times as part of a stream.

func (*StreamData) Descriptor

func (*StreamData) Descriptor() ([]byte, []int)

func (*StreamData) Marshal

func (m *StreamData) Marshal() (data []byte, err error)

func (*StreamData) MarshalTo

func (m *StreamData) MarshalTo(data []byte) (int, error)

func (*StreamData) ProtoMessage

func (*StreamData) ProtoMessage()

func (*StreamData) Reset

func (m *StreamData) Reset()

func (*StreamData) Size

func (m *StreamData) Size() (n int)

func (*StreamData) String

func (m *StreamData) String() string

func (*StreamData) Unmarshal

func (m *StreamData) Unmarshal(data []byte) error

type StreamDecoder

type StreamDecoder struct {
	// contains filtered or unexported fields
}

StreamDecoder converts a sequence of StreamMessage to EncDatumRows.

Sample usage:

sd := StreamDecoder{}
var row sqlbase.EncDatumRow
for each message in stream {
    err := sd.AddMessage(msg)
    if err != nil { ... }
    for {
        row, err := sd.GetRow(row)
        if err != nil { ... }
        if decoded != nil {
            break
        }
        // Use <row>
        ...
    }
}

AddMessage can be called multiple times before getting the rows, but this will cause data to accumulate internally.

func (*StreamDecoder) AddMessage

func (sd *StreamDecoder) AddMessage(msg *StreamMessage) error

AddMessage adds the data in a StreamMessage to the decoder.

The StreamDecoder may keep a reference to msg.Data.RawBytes until all the rows in the message are retrieved with GetRow.

func (*StreamDecoder) GetRow

func (sd *StreamDecoder) GetRow(rowBuf sqlbase.EncDatumRow) (sqlbase.EncDatumRow, error)

GetRow returns a row of EncDatums received in the stream. A row buffer can be provided optionally. Returns nil if there are no more rows received so far.

func (*StreamDecoder) IsDone

func (sd *StreamDecoder) IsDone() (bool, error)

IsDone returns true if all the rows were returned and the stream trailer was received (in which case any error in the trailer is returned as well).

type StreamEncoder

type StreamEncoder struct {
	// contains filtered or unexported fields
}

StreamEncoder converts EncDatum rows into a sequence of StreamMessage.

Sample usage:

se := StreamEncoder{}

for {
    for ... {
       err := se.AddRow(...)
       ...
    }
    msg := se.FormMessage(false, nil)
    // Send out message.
    ...
}
msg := se.FormMessage(true, nil)
// Send out final message
...

func (*StreamEncoder) AddRow

func (se *StreamEncoder) AddRow(row sqlbase.EncDatumRow) error

AddRow encodes a row.

func (*StreamEncoder) FormMessage

func (se *StreamEncoder) FormMessage(final bool, trailerErr error) *StreamMessage

FormMessage populates a message containing the rows added since the last call to FormMessage. The returned StreamMessage should be treated as immutable. If final is true, a message trailer is populated with the given error.

type StreamEndpointSpec

type StreamEndpointSpec struct {
	// If set, both ends of the stream are part of the flow on this machine. There
	// will be a corresponding endpoint with the same local stream ID.
	LocalStreamID *LocalStreamID `protobuf:"varint,1,opt,name=local_stream_id,json=localStreamId,casttype=LocalStreamID" json:"local_stream_id,omitempty"`
	// If set, the stream crosses flows; the endpoint is a mailbox.
	Mailbox *MailboxSpec `protobuf:"bytes,2,opt,name=mailbox" json:"mailbox,omitempty"`
}

StreamEndpointSpec describes one of the endpoints (input or output) of a physical stream.

func (*StreamEndpointSpec) Descriptor

func (*StreamEndpointSpec) Descriptor() ([]byte, []int)

func (*StreamEndpointSpec) GetValue

func (this *StreamEndpointSpec) GetValue() interface{}

func (*StreamEndpointSpec) Marshal

func (m *StreamEndpointSpec) Marshal() (data []byte, err error)

func (*StreamEndpointSpec) MarshalTo

func (m *StreamEndpointSpec) MarshalTo(data []byte) (int, error)

func (*StreamEndpointSpec) ProtoMessage

func (*StreamEndpointSpec) ProtoMessage()

func (*StreamEndpointSpec) Reset

func (m *StreamEndpointSpec) Reset()

func (*StreamEndpointSpec) SetValue

func (this *StreamEndpointSpec) SetValue(value interface{}) bool

func (*StreamEndpointSpec) Size

func (m *StreamEndpointSpec) Size() (n int)

func (*StreamEndpointSpec) String

func (m *StreamEndpointSpec) String() string

func (*StreamEndpointSpec) Unmarshal

func (m *StreamEndpointSpec) Unmarshal(data []byte) error

type StreamHeader

type StreamHeader struct {
	FlowID   github_com_cockroachdb_cockroach_util_uuid.UUID `protobuf:"bytes,1,opt,name=flow_id,json=flowId,customtype=github.com/cockroachdb/cockroach/util/uuid.UUID" json:"flow_id"`
	StreamID StreamID                                        `protobuf:"varint,2,opt,name=stream_id,json=streamId,casttype=StreamID" json:"stream_id"`
	// There is one DatumInfo for each element in a row.
	Info []DatumInfo `protobuf:"bytes,3,rep,name=info" json:"info"`
}

StreamHeader is a message that is sent once at the beginning of a stream.

func (*StreamHeader) Descriptor

func (*StreamHeader) Descriptor() ([]byte, []int)

func (*StreamHeader) Marshal

func (m *StreamHeader) Marshal() (data []byte, err error)

func (*StreamHeader) MarshalTo

func (m *StreamHeader) MarshalTo(data []byte) (int, error)

func (*StreamHeader) ProtoMessage

func (*StreamHeader) ProtoMessage()

func (*StreamHeader) Reset

func (m *StreamHeader) Reset()

func (*StreamHeader) Size

func (m *StreamHeader) Size() (n int)

func (*StreamHeader) String

func (m *StreamHeader) String() string

func (*StreamHeader) Unmarshal

func (m *StreamHeader) Unmarshal(data []byte) error

type StreamID

type StreamID int

StreamID identifies a stream that crosses machine boundaries. The identifier can only be used in the context of a specific flow.

type StreamMessage

type StreamMessage struct {
	// Header is present in the first message.
	Header *StreamHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
	// Data is present in all messages except possibly the first and last.
	Data StreamData `protobuf:"bytes,2,opt,name=data" json:"data"`
	// Trailer is present in the last message.
	Trailer *StreamTrailer `protobuf:"bytes,3,opt,name=trailer" json:"trailer,omitempty"`
}

func (*StreamMessage) Descriptor

func (*StreamMessage) Descriptor() ([]byte, []int)

func (*StreamMessage) Marshal

func (m *StreamMessage) Marshal() (data []byte, err error)

func (*StreamMessage) MarshalTo

func (m *StreamMessage) MarshalTo(data []byte) (int, error)

func (*StreamMessage) ProtoMessage

func (*StreamMessage) ProtoMessage()

func (*StreamMessage) Reset

func (m *StreamMessage) Reset()

func (*StreamMessage) Size

func (m *StreamMessage) Size() (n int)

func (*StreamMessage) String

func (m *StreamMessage) String() string

func (*StreamMessage) Unmarshal

func (m *StreamMessage) Unmarshal(data []byte) error

type StreamMsg

type StreamMsg struct {
	// Only one of these fields will be set.
	Row sqlbase.EncDatumRow
	Err error
}

StreamMsg is the message used in the channels that implement local physical streams.

type StreamTrailer

type StreamTrailer struct {
	Error *cockroach_roachpb2.Error `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"`
}

StreamTrailer is a message that is sent once at the end of a stream.

func (*StreamTrailer) Descriptor

func (*StreamTrailer) Descriptor() ([]byte, []int)

func (*StreamTrailer) Marshal

func (m *StreamTrailer) Marshal() (data []byte, err error)

func (*StreamTrailer) MarshalTo

func (m *StreamTrailer) MarshalTo(data []byte) (int, error)

func (*StreamTrailer) ProtoMessage

func (*StreamTrailer) ProtoMessage()

func (*StreamTrailer) Reset

func (m *StreamTrailer) Reset()

func (*StreamTrailer) Size

func (m *StreamTrailer) Size() (n int)

func (*StreamTrailer) String

func (m *StreamTrailer) String() string

func (*StreamTrailer) Unmarshal

func (m *StreamTrailer) Unmarshal(data []byte) error

type TableReaderSpan

type TableReaderSpan struct {
	// TODO(radu): the dist_sql APIs should be agnostic to how we map tables to
	// KVs. The span should be described as starting and ending lists of values
	// for a prefix of the index columns, along with inclusive/exclusive flags.
	Span cockroach_roachpb1.Span `protobuf:"bytes,1,opt,name=span" json:"span"`
}

func (*TableReaderSpan) Descriptor

func (*TableReaderSpan) Descriptor() ([]byte, []int)

func (*TableReaderSpan) Marshal

func (m *TableReaderSpan) Marshal() (data []byte, err error)

func (*TableReaderSpan) MarshalTo

func (m *TableReaderSpan) MarshalTo(data []byte) (int, error)

func (*TableReaderSpan) ProtoMessage

func (*TableReaderSpan) ProtoMessage()

func (*TableReaderSpan) Reset

func (m *TableReaderSpan) Reset()

func (*TableReaderSpan) Size

func (m *TableReaderSpan) Size() (n int)

func (*TableReaderSpan) String

func (m *TableReaderSpan) String() string

func (*TableReaderSpan) Unmarshal

func (m *TableReaderSpan) Unmarshal(data []byte) error

type TableReaderSpec

type TableReaderSpec struct {
	Table cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"`
	// If 0, we use the primary index. If non-zero, we use the index_idx-th index,
	// i.e. table.indexes[index_idx-1]
	IndexIdx uint32            `protobuf:"varint,2,opt,name=index_idx,json=indexIdx" json:"index_idx"`
	Reverse  bool              `protobuf:"varint,3,opt,name=reverse" json:"reverse"`
	Spans    []TableReaderSpan `protobuf:"bytes,4,rep,name=spans" json:"spans"`
	// The filter expression references the columns in the table (table.columns)
	// via $0, $1, etc. If a secondary index is used, the columns that are not
	// available as part of the index cannot be referenced.
	Filter Expression `protobuf:"bytes,5,opt,name=filter" json:"filter"`
	// The table reader will only produce values for these columns, referenced by
	// their indices in table.columns.
	OutputColumns []uint32 `protobuf:"varint,6,rep,packed,name=output_columns,json=outputColumns" json:"output_columns,omitempty"`
	// If nonzero, the table reader only needs to return this many rows.
	HardLimit int64 `protobuf:"varint,8,opt,name=hard_limit,json=hardLimit" json:"hard_limit"`
	// The soft limit is a hint for how many rows the consumer of the table reader
	// output might need. If both the hard limit and the soft limit are set, the
	// soft limit must be lower than the hard limit.
	SoftLimit int64 `protobuf:"varint,7,opt,name=soft_limit,json=softLimit" json:"soft_limit"`
}

TableReaderSpec is the specification for a table reader. A table reader performs KV operations to retrieve rows for a table and outputs the desired columns of the rows that pass a filter expression.

func (*TableReaderSpec) Descriptor

func (*TableReaderSpec) Descriptor() ([]byte, []int)

func (*TableReaderSpec) Marshal

func (m *TableReaderSpec) Marshal() (data []byte, err error)

func (*TableReaderSpec) MarshalTo

func (m *TableReaderSpec) MarshalTo(data []byte) (int, error)

func (*TableReaderSpec) ProtoMessage

func (*TableReaderSpec) ProtoMessage()

func (*TableReaderSpec) Reset

func (m *TableReaderSpec) Reset()

func (*TableReaderSpec) Size

func (m *TableReaderSpec) Size() (n int)

func (*TableReaderSpec) String

func (m *TableReaderSpec) String() string

func (*TableReaderSpec) Unmarshal

func (m *TableReaderSpec) Unmarshal(data []byte) error

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL