From 5afec705d169e4d87a9a7ab59fda52969bf8a0bc Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Thu, 30 Jun 2022 18:28:00 +0100 Subject: [PATCH] Improve and fix error messages (#443) Simplify, improve and make error messages consistent. Make sure we return all proper user errors and not internal error. Create errors SQL test to test many errors --- api/server.go | 6 +- client/client.go | 3 +- cluster/dragon/dragon.go | 2 +- command/command.go | 2 +- command/create_index_command.go | 10 +- command/create_mv_command.go | 7 +- command/create_source_command.go | 48 +- command/drop_index_command.go | 3 + command/parser/ast.go | 16 +- conf/conf_test.go | 70 +- errors/errors.go | 71 +- meta/meta.go | 12 +- parplan/benchmarks_test.go | 10 +- parplan/logical_plan_test.go | 2 +- parplan/parser.go | 8 +- parplan/parser_test.go | 2 +- parplan/physical_plan_test.go | 16 +- parplan/planner.go | 22 +- pull/engine.go | 16 +- pull/exec/limit.go | 6 +- pull/exec_builder.go | 4 +- push/engine.go | 6 +- push/exec_builder.go | 4 +- push/source/source.go | 17 +- remoting/remoting_test.go | 10 +- sqltest/sql_test_runner.go | 45 +- sqltest/testdata/basic_mv_test_out.txt | 6 +- sqltest/testdata/basic_source_test_out.txt | 4 +- sqltest/testdata/cascading_mvs_test_out.txt | 10 +- sqltest/testdata/compound_pk_test_out.txt | 4 +- sqltest/testdata/describe_test_out.txt | 6 +- sqltest/testdata/errors_test_data.txt | 11 + sqltest/testdata/errors_test_out.txt | 1102 +++++++++++++++++ sqltest/testdata/errors_test_script.txt | 958 ++++++++++++++ sqltest/testdata/index_test_out.txt | 12 +- .../invalid_create_source_test_out.txt | 7 +- .../invalid_create_source_test_script.txt | 2 +- sqltest/testdata/multiple_mv_test_out.txt | 10 +- sqltest/testdata/multiple_source_test_out.txt | 10 +- sqltest/testdata/on_delete_test_out.txt | 10 +- sqltest/testdata/pull_limit_test_out.txt | 16 +- sqltest/testdata/show_schemas_test_out.txt | 10 +- sqltest/testdata/show_schemas_test_script.txt | 10 +- .../source_ingest_filter_test_out.txt | 2 +- sqltest/testdata/use_test_out.txt | 16 +- 45 files changed, 2376 insertions(+), 248 deletions(-) create mode 100644 sqltest/testdata/errors_test_data.txt create mode 100644 sqltest/testdata/errors_test_out.txt create mode 100644 sqltest/testdata/errors_test_script.txt diff --git a/api/server.go b/api/server.go index c6af5eff..7015f582 100644 --- a/api/server.go +++ b/api/server.go @@ -4,6 +4,7 @@ package api import ( "context" "fmt" + "github.com/alecthomas/participle/v2" "github.com/squareup/pranadb/pull/exec" "net" "sync" @@ -151,11 +152,14 @@ func (s *Server) ExecuteStatement(in *service.ExecuteStatementRequest, func (s *Server) doExecuteStatement(executor exec.PullExecutor, batchSize int, err error, stream service.PranaDBService_ExecuteStatementServer) error { if err != nil { - log.Errorf("failed to execute statement %+v", err) var perr errors.PranaError if errors.As(err, &perr) { return perr } + var participleErr participle.Error + if errors.As(err, &participleErr) { + return errors.NewInvalidStatementError(participleErr.Error()) + } return common.LogInternalError(err) } diff --git a/client/client.go b/client/client.go index 3b366dd6..872f8099 100644 --- a/client/client.go +++ b/client/client.go @@ -25,6 +25,7 @@ const ( minLineWidth = 10 minColWidth = 5 maxLineWidthPropName = "max_line_width" + maxLineWidth = 10000 ) // Client is a simple client used for executing statements against PranaDB, it used by the CLI and elsewhere @@ -116,7 +117,7 @@ func (c *Client) handleSetCommand(statement string) error { if propName := parts[1]; propName == maxLineWidthPropName { propVal := parts[2] width, err := strconv.Atoi(propVal) - if err != nil || width < minLineWidth { + if err != nil || width < minLineWidth || width > maxLineWidth { return errors.Errorf("Invalid %s value: %s", maxLineWidthPropName, propVal) } c.lock.Lock() diff --git a/cluster/dragon/dragon.go b/cluster/dragon/dragon.go index 2d211ad1..589f5243 100644 --- a/cluster/dragon/dragon.go +++ b/cluster/dragon/dragon.go @@ -195,7 +195,7 @@ func (d *Dragon) ExecuteRemotePullQuery(queryInfo *cluster.QueryExecutionInfo, r if bytes[0] == 0 { if time.Now().Sub(start) > pullQueryRetryTimeout { msg := string(bytes[1:]) - return nil, errors.Errorf("failed to execute remote query %s %v", queryInfo.Query, msg) + return nil, errors.Errorf("failed to execute remote query %v", msg) } // Retry - the pull engine might not be fully started.... this can occur as the pull engine is not fully // initialised until after the cluster is active diff --git a/command/command.go b/command/command.go index f4bdaf26..12a18cf2 100644 --- a/command/command.go +++ b/command/command.go @@ -240,7 +240,7 @@ func (e *Executor) execDescribe(execCtx *execctx.ExecutionContext, tableName str return nil, errors.WithStack(err) } if rows.RowCount() == 0 { - return nil, errors.NewUnknownSourceOrMaterializedViewError(execCtx.Schema.Name, tableName) + return nil, errors.NewUnknownTableError(execCtx.Schema.Name, tableName) } if rows.RowCount() != 1 { panic(fmt.Sprintf("multiple matches for table: '%s.%s'", execCtx.Schema.Name, tableName)) diff --git a/command/create_index_command.go b/command/create_index_command.go index 5e6efe7a..25231fda 100644 --- a/command/create_index_command.go +++ b/command/create_index_command.go @@ -147,7 +147,7 @@ func (c *CreateIndexCommand) getIndexInfo(ast *parser.CreateIndex) (*common.Inde if !ok { tab, ok = c.e.metaController.GetMaterializedView(c.SchemaName(), ast.TableName) if !ok { - return nil, errors.NewUnknownSourceOrMaterializedViewError(c.SchemaName(), ast.TableName) + return nil, errors.NewUnknownTableError(c.SchemaName(), ast.TableName) } } tabInfo := tab.GetTableInfo() @@ -164,12 +164,18 @@ func (c *CreateIndexCommand) getIndexInfo(ast *parser.CreateIndex) (*common.Inde colMap[colName] = colIndex } indexCols := make([]int, len(ast.ColumnNames)) + indexColMap := make(map[int]struct{}, len(ast.ColumnNames)) for i, colName := range ast.ColumnNames { colIndex, ok := colMap[colName.Name] if !ok { - return nil, errors.NewUnknownIndexColumn(c.SchemaName(), ast.TableName, colName.Name) + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Unknown column %s in %s.%s", + colName.Name, c.SchemaName(), ast.TableName) } indexCols[i] = colIndex + indexColMap[colIndex] = struct{}{} + } + if len(indexColMap) != len(ast.ColumnNames) { + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Index cannot contain same column multiple times") } info := &common.IndexInfo{ SchemaName: c.SchemaName(), diff --git a/command/create_mv_command.go b/command/create_mv_command.go index 38c9f45f..4a866604 100644 --- a/command/create_mv_command.go +++ b/command/create_mv_command.go @@ -95,10 +95,11 @@ func (c *CreateMVCommand) Before() error { return errors.WithStack(err) } c.mv = mv - _, ok := c.e.metaController.GetMaterializedView(mv.Info.SchemaName, mv.Info.Name) - if ok { - return errors.NewMaterializedViewAlreadyExistsError(mv.Info.SchemaName, mv.Info.Name) + + if err := c.e.metaController.ExistsMvOrSource(c.schema, mv.Info.Name); err != nil { + return err } + rows, err := c.e.pullEngine.ExecuteQuery("sys", fmt.Sprintf("select id from tables where schema_name='%s' and name='%s' and kind='%s'", c.mv.Info.SchemaName, c.mv.Info.Name, meta.TableKindMaterializedView)) if err != nil { diff --git a/command/create_source_command.go b/command/create_source_command.go index 48dd7245..ed70a38c 100644 --- a/command/create_source_command.go +++ b/command/create_source_command.go @@ -76,9 +76,9 @@ func (c *CreateSourceCommand) Before() error { } func (c *CreateSourceCommand) validate() error { - _, ok := c.e.metaController.GetSource(c.schemaName, c.sourceInfo.Name) - if ok { - return errors.NewSourceAlreadyExistsError(c.schemaName, c.sourceInfo.Name) + schema := c.e.metaController.GetOrCreateSchema(c.schemaName) + if err := c.e.metaController.ExistsMvOrSource(schema, c.sourceInfo.Name); err != nil { + return err } rows, err := c.e.pullEngine.ExecuteQuery("sys", fmt.Sprintf("select id from tables where schema_name='%s' and name='%s' and kind='%s'", c.sourceInfo.SchemaName, c.sourceInfo.Name, meta.TableKindSource)) @@ -97,18 +97,18 @@ func (c *CreateSourceCommand) validate() error { } _, err := c.e.protoRegistry.FindDescriptorByName(protoreflect.FullName(enc.SchemaName)) if err != nil { - return errors.NewPranaErrorf(errors.UnknownTopicEncoding, "proto message %q not registered", enc.SchemaName) + return errors.NewPranaErrorf(errors.InvalidStatement, "Proto message %q not registered", enc.SchemaName) } } for _, sel := range topicInfo.ColSelectors { if sel.MetaKey == nil && len(sel.Selector) == 0 { - return errors.NewPranaErrorf(errors.InvalidSelector, "invalid column selector %q", sel) + return errors.NewPranaErrorf(errors.InvalidStatement, "Invalid column selector %q", sel) } if sel.MetaKey != nil { f := *sel.MetaKey if !(f == "header" || f == "key" || f == "timestamp") { - return errors.NewPranaErrorf(errors.InvalidSelector, `invalid metadata key in column selector %q. Valid values are "header", "key", "timestamp".`, sel) + return errors.NewPranaErrorf(errors.InvalidStatement, `Invalid metadata key in column selector %q. Valid values are "header", "key", "timestamp".`, sel) } } } @@ -210,7 +210,7 @@ func (c *CreateSourceCommand) getSourceInfo(ast *parser.CreateSource) (*common.S for _, pk := range option.PrimaryKey { index, ok := colIndex[pk] if !ok { - return nil, errors.Errorf("invalid primary key column %q", option.PrimaryKey) + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Invalid primary key column %q", option.PrimaryKey) } pkCols = append(pkCols, index) } @@ -232,17 +232,17 @@ func (c *CreateSourceCommand) getSourceInfo(ast *parser.CreateSource) (*common.S case opt.HeaderEncoding != "": headerEncoding = common.KafkaEncodingFromString(opt.HeaderEncoding) if headerEncoding.Encoding == common.EncodingUnknown { - return nil, errors.NewPranaErrorf(errors.UnknownTopicEncoding, "Unknown topic encoding %s", opt.HeaderEncoding) + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Unknown topic encoding %s", opt.HeaderEncoding) } case opt.KeyEncoding != "": keyEncoding = common.KafkaEncodingFromString(opt.KeyEncoding) if keyEncoding.Encoding == common.EncodingUnknown { - return nil, errors.NewPranaErrorf(errors.UnknownTopicEncoding, "Unknown topic encoding %s", opt.KeyEncoding) + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Unknown topic encoding %s", opt.KeyEncoding) } case opt.ValueEncoding != "": valueEncoding = common.KafkaEncodingFromString(opt.ValueEncoding) if valueEncoding.Encoding == common.EncodingUnknown { - return nil, errors.NewPranaErrorf(errors.UnknownTopicEncoding, "Unknown topic encoding %s", opt.ValueEncoding) + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Unknown topic encoding %s", opt.ValueEncoding) } case opt.IngestFilter != "": ingestFilter = opt.IngestFilter @@ -264,24 +264,34 @@ func (c *CreateSourceCommand) getSourceInfo(ast *parser.CreateSource) (*common.S } } if headerEncoding == common.KafkaEncodingUnknown { - return nil, errors.NewPranaError(errors.InvalidStatement, "headerEncoding is required") + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "headerEncoding is required") } if keyEncoding == common.KafkaEncodingUnknown { - return nil, errors.NewPranaError(errors.InvalidStatement, "keyEncoding is required") + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "keyEncoding is required") } if valueEncoding == common.KafkaEncodingUnknown { - return nil, errors.NewPranaError(errors.InvalidStatement, "valueEncoding is required") + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "valueEncoding is required") } if brokerName == "" { - return nil, errors.NewPranaError(errors.InvalidStatement, "brokerName is required") + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "brokerName is required") } if topicName == "" { - return nil, errors.NewPranaError(errors.InvalidStatement, "topicName is required") + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "topicName is required") } - lc := len(colSelectors) - if lc > 0 && lc != len(colTypes) { - return nil, errors.NewPranaErrorf(errors.WrongNumberColumnSelectors, - "Number of column selectors (%d) must match number of columns (%d)", lc, len(colTypes)) + if len(colSelectors) != len(colTypes) { + return nil, errors.NewPranaErrorf(errors.InvalidStatement, + "Number of column selectors (%d) must match number of columns (%d)", len(colSelectors), len(colTypes)) + } + if len(pkCols) == 0 { + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Primary key is required") + } + + pkMap := make(map[int]struct{}, len(pkCols)) + for _, pkCol := range pkCols { + pkMap[pkCol] = struct{}{} + } + if len(pkMap) != len(pkCols) { + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Primary key cannot contain same column multiple times") } topicInfo := &common.TopicInfo{ diff --git a/command/drop_index_command.go b/command/drop_index_command.go index 078ad0bd..701f9fe6 100644 --- a/command/drop_index_command.go +++ b/command/drop_index_command.go @@ -147,6 +147,9 @@ func (c *DropIndexCommand) getIndexInfo() (*common.IndexInfo, error) { c.indexName = ast.Drop.Name c.tableName = ast.Drop.TableName } + if c.tableName == "" { + return nil, errors.NewInvalidStatementError("Drop index requires a table") + } indexInfo, ok := c.e.metaController.GetIndex(c.schemaName, c.tableName, c.indexName) if !ok { return nil, errors.NewUnknownIndexError(c.schemaName, c.tableName, c.indexName) diff --git a/command/parser/ast.go b/command/parser/ast.go index 4b7aa491..6b4a5787 100644 --- a/command/parser/ast.go +++ b/command/parser/ast.go @@ -73,13 +73,27 @@ func (c *ColumnDef) ToColumnType() (common.ColumnType, error) { switch c.Type { case common.TypeDecimal: if len(c.Parameters) != 2 { - return common.ColumnType{}, participle.Errorf(c.Pos, "expected DECIMAL(precision, scale)") + return common.ColumnType{}, participle.Errorf(c.Pos, "Expected DECIMAL(precision, scale)") + } + prec := c.Parameters[0] + scale := c.Parameters[1] + if prec > 65 || prec < 1 { + return common.ColumnType{}, participle.Errorf(c.Pos, "Decimal precision must be > 0 and <= 65") + } + if scale > 30 || scale < 0 { + return common.ColumnType{}, participle.Errorf(c.Pos, "decimal scale must be >= 0 and <= 30") + } + if scale > prec { + return common.ColumnType{}, participle.Errorf(c.Pos, "Decimal scale must be <= precision") } return common.NewDecimalColumnType(c.Parameters[0], c.Parameters[1]), nil case common.TypeTimestamp: var fsp int8 = DefaultFSP if len(c.Parameters) == 1 { fsp = int8(c.Parameters[0]) + if fsp < 0 || fsp > 6 { + return common.ColumnType{}, participle.Errorf(c.Pos, "Timestamp fsp must be >= 0 and <= 6") + } } return common.NewTimestampColumnType(fsp), nil default: diff --git a/conf/conf_test.go b/conf/conf_test.go index aa5a5c9a..4c1fa992 100644 --- a/conf/conf_test.go +++ b/conf/conf_test.go @@ -259,41 +259,41 @@ func invalidRaftElectionRTTTooSmall() Config { } var invalidConfigs = []configPair{ - {"PDB0004 - Invalid configuration: NodeID must be >= 0", invalidNodeIDConf()}, - {"PDB0004 - Invalid configuration: NumShards must be >= 1", invalidNumShardsConf()}, - {"PDB0004 - Invalid configuration: DataDir must be specified", invalidDatadirConf()}, - {"PDB0004 - Invalid configuration: KafkaBrokers must be specified", missingKafkaBrokersConf()}, - {"PDB0004 - Invalid configuration: KafkaBroker testbroker, invalid ClientType, must be 1 or 2", invalidBrokerClientTypeConf()}, - {"PDB0004 - Invalid configuration: RemotingHeartbeatInterval must be >= 1000000000", invalidRemotingHeartbeatInterval()}, - {"PDB0004 - Invalid configuration: RemotingHeartbeatTimeout must be >= 1000000", invalidRemotingHeartbeatTimeout()}, - {"PDB0004 - Invalid configuration: APIServerListenAddresses must be specified", invalidAPIServerListenAddress()}, - {"PDB0004 - Invalid configuration: NodeID must be in the range 0 (inclusive) to len(RaftAddresses) (exclusive)", NodeIDOutOfRangeConf()}, - {"PDB0004 - Invalid configuration: ReplicationFactor must be >= 3", invalidReplicationFactorConfig()}, - {"PDB0004 - Invalid configuration: Number of RaftAddresses must be >= ReplicationFactor", invalidRaftAddressesConfig()}, - {"PDB0004 - Invalid configuration: Number of RaftAddresses must be same as number of NotifListenerAddresses", raftAndNotifListenerAddressedDifferentLengthConfig()}, - {"PDB0004 - Invalid configuration: Number of RaftAddresses must be same as number of APIServerListenAddresses", raftAndAPIServerListenerAddressedDifferentLengthConfig()}, - {"PDB0004 - Invalid configuration: DataSnapshotEntries must be >= 10", invalidDataSnapshotEntries()}, - {"PDB0004 - Invalid configuration: DataCompactionOverhead must be >= 5", invalidDataCompactionOverhead()}, - {"PDB0004 - Invalid configuration: SequenceSnapshotEntries must be >= 10", invalidSequenceSnapshotEntries()}, - {"PDB0004 - Invalid configuration: SequenceCompactionOverhead must be >= 5", invalidSequenceCompactionOverhead()}, - {"PDB0004 - Invalid configuration: LocksSnapshotEntries must be >= 10", invalidLocksSnapshotEntries()}, - {"PDB0004 - Invalid configuration: LocksCompactionOverhead must be >= 5", invalidLocksCompactionOverhead()}, - {"PDB0004 - Invalid configuration: DataSnapshotEntries must be >= DataCompactionOverhead", dataCompactionGreaterThanDataSnapshotEntries()}, - {"PDB0004 - Invalid configuration: SequenceSnapshotEntries must be >= SequenceCompactionOverhead", sequenceCompactionGreaterThanDataSnapshotEntries()}, - {"PDB0004 - Invalid configuration: LocksSnapshotEntries must be >= LocksCompactionOverhead", locksCompactionGreaterThanDataSnapshotEntries()}, - {"PDB0004 - Invalid configuration: LifeCycleListenAddress must be specified", invalidLifecycleListenAddress()}, - {"PDB0004 - Invalid configuration: StartupEndpointPath must be specified", invalidStartupEndpointPath()}, - {"PDB0004 - Invalid configuration: LiveEndpointPath must be specified", invalidLiveEndpointPath()}, - {"PDB0004 - Invalid configuration: ReadyEndpointPath must be specified", invalidReadyEndpointPath()}, - {"PDB0004 - Invalid configuration: GlobalIngestLimitRowsPerSec must be > 0 or -1", invalidGlobalIngestLimitRowsPerSecZero()}, - {"PDB0004 - Invalid configuration: GlobalIngestLimitRowsPerSec must be > 0 or -1", invalidGlobalIngestLimitRowsPerNegative()}, - {"PDB0004 - Invalid configuration: RaftRTTMs must be > 0", invalidRaftRTTMsZero()}, - {"PDB0004 - Invalid configuration: RaftRTTMs must be > 0", invalidRaftRTTMsNegative()}, - {"PDB0004 - Invalid configuration: RaftHeartbeatRTT must be > 0", invalidRaftHeartbeatRTTZero()}, - {"PDB0004 - Invalid configuration: RaftHeartbeatRTT must be > 0", invalidRaftHeartbeatRTTNegative()}, - {"PDB0004 - Invalid configuration: RaftElectionRTT must be > 0", invalidRaftElectionRTTZero()}, - {"PDB0004 - Invalid configuration: RaftElectionRTT must be > 0", invalidRaftElectionRTTNegative()}, - {"PDB0004 - Invalid configuration: RaftElectionRTT must be > 2 * RaftHeartbeatRTT", invalidRaftElectionRTTTooSmall()}, + {"PDB3000 - Invalid configuration: NodeID must be >= 0", invalidNodeIDConf()}, + {"PDB3000 - Invalid configuration: NumShards must be >= 1", invalidNumShardsConf()}, + {"PDB3000 - Invalid configuration: DataDir must be specified", invalidDatadirConf()}, + {"PDB3000 - Invalid configuration: KafkaBrokers must be specified", missingKafkaBrokersConf()}, + {"PDB3000 - Invalid configuration: KafkaBroker testbroker, invalid ClientType, must be 1 or 2", invalidBrokerClientTypeConf()}, + {"PDB3000 - Invalid configuration: RemotingHeartbeatInterval must be >= 1000000000", invalidRemotingHeartbeatInterval()}, + {"PDB3000 - Invalid configuration: RemotingHeartbeatTimeout must be >= 1000000", invalidRemotingHeartbeatTimeout()}, + {"PDB3000 - Invalid configuration: APIServerListenAddresses must be specified", invalidAPIServerListenAddress()}, + {"PDB3000 - Invalid configuration: NodeID must be in the range 0 (inclusive) to len(RaftAddresses) (exclusive)", NodeIDOutOfRangeConf()}, + {"PDB3000 - Invalid configuration: ReplicationFactor must be >= 3", invalidReplicationFactorConfig()}, + {"PDB3000 - Invalid configuration: Number of RaftAddresses must be >= ReplicationFactor", invalidRaftAddressesConfig()}, + {"PDB3000 - Invalid configuration: Number of RaftAddresses must be same as number of NotifListenerAddresses", raftAndNotifListenerAddressedDifferentLengthConfig()}, + {"PDB3000 - Invalid configuration: Number of RaftAddresses must be same as number of APIServerListenAddresses", raftAndAPIServerListenerAddressedDifferentLengthConfig()}, + {"PDB3000 - Invalid configuration: DataSnapshotEntries must be >= 10", invalidDataSnapshotEntries()}, + {"PDB3000 - Invalid configuration: DataCompactionOverhead must be >= 5", invalidDataCompactionOverhead()}, + {"PDB3000 - Invalid configuration: SequenceSnapshotEntries must be >= 10", invalidSequenceSnapshotEntries()}, + {"PDB3000 - Invalid configuration: SequenceCompactionOverhead must be >= 5", invalidSequenceCompactionOverhead()}, + {"PDB3000 - Invalid configuration: LocksSnapshotEntries must be >= 10", invalidLocksSnapshotEntries()}, + {"PDB3000 - Invalid configuration: LocksCompactionOverhead must be >= 5", invalidLocksCompactionOverhead()}, + {"PDB3000 - Invalid configuration: DataSnapshotEntries must be >= DataCompactionOverhead", dataCompactionGreaterThanDataSnapshotEntries()}, + {"PDB3000 - Invalid configuration: SequenceSnapshotEntries must be >= SequenceCompactionOverhead", sequenceCompactionGreaterThanDataSnapshotEntries()}, + {"PDB3000 - Invalid configuration: LocksSnapshotEntries must be >= LocksCompactionOverhead", locksCompactionGreaterThanDataSnapshotEntries()}, + {"PDB3000 - Invalid configuration: LifeCycleListenAddress must be specified", invalidLifecycleListenAddress()}, + {"PDB3000 - Invalid configuration: StartupEndpointPath must be specified", invalidStartupEndpointPath()}, + {"PDB3000 - Invalid configuration: LiveEndpointPath must be specified", invalidLiveEndpointPath()}, + {"PDB3000 - Invalid configuration: ReadyEndpointPath must be specified", invalidReadyEndpointPath()}, + {"PDB3000 - Invalid configuration: GlobalIngestLimitRowsPerSec must be > 0 or -1", invalidGlobalIngestLimitRowsPerSecZero()}, + {"PDB3000 - Invalid configuration: GlobalIngestLimitRowsPerSec must be > 0 or -1", invalidGlobalIngestLimitRowsPerNegative()}, + {"PDB3000 - Invalid configuration: RaftRTTMs must be > 0", invalidRaftRTTMsZero()}, + {"PDB3000 - Invalid configuration: RaftRTTMs must be > 0", invalidRaftRTTMsNegative()}, + {"PDB3000 - Invalid configuration: RaftHeartbeatRTT must be > 0", invalidRaftHeartbeatRTTZero()}, + {"PDB3000 - Invalid configuration: RaftHeartbeatRTT must be > 0", invalidRaftHeartbeatRTTNegative()}, + {"PDB3000 - Invalid configuration: RaftElectionRTT must be > 0", invalidRaftElectionRTTZero()}, + {"PDB3000 - Invalid configuration: RaftElectionRTT must be > 0", invalidRaftElectionRTTNegative()}, + {"PDB3000 - Invalid configuration: RaftElectionRTT must be > 2 * RaftHeartbeatRTT", invalidRaftElectionRTTTooSmall()}, } func TestValidate(t *testing.T) { diff --git a/errors/errors.go b/errors/errors.go index 8167a140..e2297ffc 100644 --- a/errors/errors.go +++ b/errors/errors.go @@ -8,36 +8,33 @@ import ( type ErrorCode int +// User errors const ( - InternalError = iota + InvalidStatement = iota + 1000 SchemaNotInUse - InvalidStatement - UnknownSessionID - InvalidConfiguration UnknownSource UnknownMaterializedView - UnknownPreparedStatement + UnknownIndex + UnknownTable SourceAlreadyExists MaterializedViewAlreadyExists + IndexAlreadyExists SourceHasChildren MaterializedViewHasChildren + InvalidParamCount +) - UnknownBrokerName - MissingKafkaBrokers - MissingTopicInfo - UnsupportedBrokerClientType - UnknownTopicEncoding - WrongNumberColumnSelectors - InvalidSelector - UnknownSourceOrMaterializedView - UnknownIndexColumn - IndexAlreadyExists - - UnknownPerfCommand - - ValueOutOfRange +// Ingest errors +const ( + ValueOutOfRange = iota + 2000 VarcharTooBig - InvalidIngestFilter +) + +// Miscellaneous errors +const ( + InvalidConfiguration = 3000 + UnknownPerfCommand = 4000 + InternalError = 5000 ) func NewInternalError(errReference string) PranaError { @@ -52,10 +49,6 @@ func NewInvalidStatementError(msg string) PranaError { return NewPranaErrorf(InvalidStatement, msg) } -func NewUnknownSessionIDError(sessionID string) PranaError { - return NewPranaErrorf(UnknownSessionID, "Unknown session ID %s", sessionID) -} - func NewInvalidConfigurationError(msg string) PranaError { return NewPranaErrorf(InvalidConfiguration, "Invalid configuration: %s", msg) } @@ -65,37 +58,21 @@ func NewUnknownSourceError(schemaName string, sourceName string) PranaError { } func NewUnknownIndexError(schemaName string, tableName string, indexName string) PranaError { - return NewPranaErrorf(UnknownSource, "Unknown index: %s.%s.%s", schemaName, tableName, indexName) + return NewPranaErrorf(UnknownIndex, "Unknown index: %s.%s.%s", schemaName, tableName, indexName) } func NewUnknownMaterializedViewError(schemaName string, mvName string) PranaError { return NewPranaErrorf(UnknownMaterializedView, "Unknown materialized view: %s.%s", schemaName, mvName) } -func NewUnknownSourceOrMaterializedViewError(schemaName string, tableName string) PranaError { - return NewPranaErrorf(UnknownSourceOrMaterializedView, "Unknown source or materialized view: %s.%s", schemaName, tableName) -} - -func NewUnknownIndexColumn(schemaName string, tableName string, columnName string) PranaError { - return NewPranaErrorf(UnknownIndexColumn, "Table %s.%s does not have a column %s", schemaName, tableName, columnName) -} - -func NewUnknownPreparedStatementError(psID int64) PranaError { - return NewPranaErrorf(UnknownPreparedStatement, "Unknown prepared statement, id: %d", psID) -} - -func NewSourceAlreadyExistsError(schemaName string, sourceName string) PranaError { - return NewPranaErrorf(SourceAlreadyExists, "Source already exists: %s.%s", schemaName, sourceName) +func NewUnknownTableError(schemaName string, tableName string) PranaError { + return NewPranaErrorf(UnknownTable, "Unknown source or materialized view: %s.%s", schemaName, tableName) } func NewIndexAlreadyExistsError(schemaName string, tableName string, indexName string) PranaError { return NewPranaErrorf(IndexAlreadyExists, "Index %s already exists on %s.%s", indexName, schemaName, tableName) } -func NewMaterializedViewAlreadyExistsError(schemaName string, materializedViewName string) PranaError { - return NewPranaErrorf(MaterializedViewAlreadyExists, "Materialized view already exists: %s.%s", schemaName, materializedViewName) -} - func NewSourceHasChildrenError(schemaName string, sourceName string, childMVs []string) PranaError { return NewPranaErrorf(SourceHasChildren, "Cannot drop source %s.%s it has the following children %s", schemaName, sourceName, getChildString(schemaName, childMVs)) } @@ -147,7 +124,13 @@ func (u PranaError) Error() string { return u.Msg } -// FIXME - fix all these - quick fixed here after port from TiDB +func MaybeConvertToPranaErrorf(err error, errorCode ErrorCode, msgFormat string, args ...interface{}) error { + var perr PranaError + if As(err, &perr) { + return err + } + return NewPranaErrorf(errorCode, msgFormat, args...) +} func Trace(err error) error { return err diff --git a/meta/meta.go b/meta/meta.go index 657732bd..ac7c5809 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -204,8 +204,16 @@ func (c *Controller) ExistsMvOrSource(schema *common.Schema, name string) error } func (c *Controller) existsTable(schema *common.Schema, name string) error { - if _, ok := schema.GetTable(name); ok { - return errors.Errorf("table with Name %s already exists in Schema %s", name, schema.Name) + if tbl, ok := schema.GetTable(name); ok { + _, isSource := tbl.(*common.SourceInfo) + _, isMV := tbl.(*common.MaterializedViewInfo) + if isSource { + return errors.NewPranaErrorf(errors.SourceAlreadyExists, "Source %s.%s already exists", schema.Name, name) + } else if isMV { + return errors.NewPranaErrorf(errors.MaterializedViewAlreadyExists, "Materialized view %s.%s already exists", schema.Name, name) + } else { + return errors.Errorf("Table %s.%s already exists", schema.Name, name) + } } return nil } diff --git a/parplan/benchmarks_test.go b/parplan/benchmarks_test.go index 7e0164a7..87e1aace 100644 --- a/parplan/benchmarks_test.go +++ b/parplan/benchmarks_test.go @@ -8,7 +8,7 @@ import ( func BenchmarkParser(b *testing.B) { for i := 0; i < b.N; i++ { parser := NewParser() - stmt, err := parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") + stmt, _, err := parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") require.NoError(b, err) require.NotNil(b, stmt) } @@ -26,7 +26,7 @@ func BenchmarkCreatePlanner(b *testing.B) { func BenchmarkPreprocess(b *testing.B) { schema := createTestSchema() planner := NewPlanner(schema) - ast, err := planner.parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") + ast, _, err := planner.parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") require.NoError(b, err) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -38,7 +38,7 @@ func BenchmarkPreprocess(b *testing.B) { func BenchmarkLogicalPlan(b *testing.B) { schema := createTestSchema() planner := NewPlanner(schema) - ast, err := planner.parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") + ast, _, err := planner.parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") require.NoError(b, err) err = planner.preprocess(ast.stmt, false) require.NoError(b, err) @@ -53,7 +53,7 @@ func BenchmarkLogicalPlan(b *testing.B) { func BenchmarkPhysicalPlan(b *testing.B) { schema := createTestSchema() planner := NewPlanner(schema) - ast, err := planner.parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") + ast, _, err := planner.parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") require.NoError(b, err) err = planner.preprocess(ast.stmt, false) require.NoError(b, err) @@ -73,7 +73,7 @@ func BenchmarkFullProcess(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { planner := NewPlanner(schema) - ast, err := planner.parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") + ast, _, err := planner.parser.Parse("select col0, col1, col2 from table1 where col1 = 123456 order by col0, col1") require.NoError(b, err) err = planner.preprocess(ast.stmt, false) require.NoError(b, err) diff --git a/parplan/logical_plan_test.go b/parplan/logical_plan_test.go index 7844b9b0..4ab49bad 100644 --- a/parplan/logical_plan_test.go +++ b/parplan/logical_plan_test.go @@ -178,7 +178,7 @@ func testLogicalPlan(t *testing.T, query string, expectedPlan string) { t.Helper() schema := createTestSchema() planner := NewPlanner(schema) - ast, err := planner.parser.Parse(query) + ast, _, err := planner.parser.Parse(query) require.NoError(t, err) err = planner.preprocess(ast.stmt, false) require.NoError(t, err) diff --git a/parplan/parser.go b/parplan/parser.go index 422e87cc..30fabf0d 100644 --- a/parplan/parser.go +++ b/parplan/parser.go @@ -21,10 +21,10 @@ type Parser struct { parser *pc_parser.Parser } -func (p *Parser) Parse(sql string) (stmt AstHandle, err error) { +func (p *Parser) Parse(sql string) (stmt AstHandle, paramCount int, err error) { stmtNodes, warns, err := p.parser.Parse(sql, charset.CharsetUTF8, "") if err != nil { - return AstHandle{}, errors.WithStack(err) + return AstHandle{}, 0, errors.WithStack(err) } if warns != nil { for _, warn := range warns { @@ -32,7 +32,7 @@ func (p *Parser) Parse(sql string) (stmt AstHandle, err error) { } } if len(stmtNodes) != 1 { - return AstHandle{}, errors.Errorf("expected 1 statement got %d", len(stmtNodes)) + return AstHandle{}, 0, errors.Errorf("expected 1 statement got %d", len(stmtNodes)) } // We gather the param marker expressions then sort them in order of where they appear in the original sql @@ -48,7 +48,7 @@ func (p *Parser) Parse(sql string) (stmt AstHandle, err error) { pme.SetOrder(i) } - return AstHandle{stmt: stmtNode}, nil + return AstHandle{stmt: stmtNode}, len(pms), nil } // AstHandle wraps the underlying TiDB ast, to avoid leaking the TiDB too much into the rest of the code diff --git a/parplan/parser_test.go b/parplan/parser_test.go index 001fa6d7..a38da0a1 100644 --- a/parplan/parser_test.go +++ b/parplan/parser_test.go @@ -8,6 +8,6 @@ func TestParse(t *testing.T) { parser := NewParser() - _, _ = parser.Parse("select t1.col1, t1.col2, t2.col3 from table1 t1 inner join table2 t2 on t1.col1 = t2.col3 order by t1.col1") + _, _, _ = parser.Parse("select t1.col1, t1.col2, t2.col3 from table1 t1 inner join table2 t2 on t1.col1 = t2.col3 order by t1.col1") } diff --git a/parplan/physical_plan_test.go b/parplan/physical_plan_test.go index b68e69c6..1d6083c9 100644 --- a/parplan/physical_plan_test.go +++ b/parplan/physical_plan_test.go @@ -182,7 +182,7 @@ func testQueryUsesTableOrIndexScan(t *testing.T, query string, tableScan bool, e } } planner := NewPlanner(schema) - physi, _, err := planner.QueryToPlan(query, false, true) + physi, _, _, err := planner.QueryToPlan(query, false, true) require.NoError(t, err) var ranges []*ranger.Range if tableScan { @@ -222,7 +222,7 @@ func testQueryUsesTableOrIndexScan(t *testing.T, query string, tableScan bool, e func TestPointGetUsesSelectForPushQuery(t *testing.T) { schema := createTestSchema() planner := NewPlanner(schema) - physi, _, err := planner.QueryToPlan("select col0, col1, col2 from table1 where col0=123", false, false) + physi, _, _, err := planner.QueryToPlan("select col0, col1, col2 from table1 where col0=123", false, false) require.NoError(t, err) sel, ok := physi.(*planner2.PhysicalSelection) require.True(t, ok) @@ -238,7 +238,7 @@ func TestSecondaryIndexLookupUsingIndexScanForPullQuery(t *testing.T) { schema, err := attachIndexToSchema(schema) require.NoError(t, err) planner := NewPlanner(schema) - physi, _, err := planner.QueryToPlan("select col2 from table1 where col2=1", false, true) + physi, _, _, err := planner.QueryToPlan("select col2 from table1 where col2=1", false, true) require.NoError(t, err) is, ok := physi.(*planner2.PhysicalIndexScan) require.True(t, ok) @@ -252,7 +252,7 @@ func TestSecondaryIndexLookupWithInUsingIndexScanWithMultipleRangesForPullQuery( schema, err := attachIndexToSchema(schema) require.NoError(t, err) planner := NewPlanner(schema) - physi, _, err := planner.QueryToPlan("select col2 from table1 where col2 in (100, 200, 300)", false, true) + physi, _, _, err := planner.QueryToPlan("select col2 from table1 where col2 in (100, 200, 300)", false, true) require.NoError(t, err) is, ok := physi.(*planner2.PhysicalIndexScan) require.True(t, ok) @@ -266,7 +266,7 @@ func TestSecondaryIndexLookupWithInUsingIndexScanWithMultipleRangesForPullQuery( func TestPrimaryKeyLookupWithInUsingTableScanWithMultipleRangesForPullQuery(t *testing.T) { schema := createTestSchema() planner := NewPlanner(schema) - physi, _, err := planner.QueryToPlan("select col0 from table1 where col0 in (100, 200, 300)", false, true) + physi, _, _, err := planner.QueryToPlan("select col0 from table1 where col0 in (100, 200, 300)", false, true) require.NoError(t, err) ts, ok := physi.(*planner2.PhysicalTableScan) require.True(t, ok) @@ -282,7 +282,7 @@ func TestSecondaryIndexMultiColumnLookupUsingIndexScanForPullQuery(t *testing.T) schema, err := attachMultiColumnIndexToSchema(schema) require.NoError(t, err) planner := NewPlanner(schema) - physi, _, err := planner.QueryToPlan("select * from table4 where col1='foo' and col2=432 and col3='bar'", false, true) + physi, _, _, err := planner.QueryToPlan("select * from table4 where col1='foo' and col2=432 and col3='bar'", false, true) require.NoError(t, err) is, ok := physi.(*planner2.PhysicalIndexScan) require.True(t, ok) @@ -301,7 +301,7 @@ func TestSecondaryIndexRangeUsingIndexScanForPullQuery(t *testing.T) { schema, err := attachIndexToSchema(schema) require.NoError(t, err) planner := NewPlanner(schema) - physi, _, err := planner.QueryToPlan("select col2 from table1 where col2 > 1", false, true) + physi, _, _, err := planner.QueryToPlan("select col2 from table1 where col2 > 1", false, true) require.NoError(t, err) is, ok := physi.(*planner2.PhysicalIndexScan) require.True(t, ok) @@ -314,7 +314,7 @@ func TestSecondaryIndexMultiColumnRangeUsingIndexScanForPullQuery(t *testing.T) schema, err := attachMultiColumnIndexToSchema(schema) require.NoError(t, err) planner := NewPlanner(schema) - physi, _, err := planner.QueryToPlan("select * from table4 where col1='foo' and col2=432 and col3>'bar'", false, true) + physi, _, _, err := planner.QueryToPlan("select * from table4 where col1='foo' and col2=432 and col3>'bar'", false, true) require.NoError(t, err) is, ok := physi.(*planner2.PhysicalIndexScan) require.True(t, ok) diff --git a/parplan/planner.go b/parplan/planner.go index 5d30a5a6..c2e32ca1 100644 --- a/parplan/planner.go +++ b/parplan/planner.go @@ -48,24 +48,32 @@ func (p *Planner) SetPSArgs(args []interface{}) { p.sessionCtx.SetArgs(args) } -func (p *Planner) Parse(query string) (AstHandle, error) { +func (p *Planner) Parse(query string) (AstHandle, int, error) { return p.parser.Parse(query) } -func (p *Planner) QueryToPlan(query string, prepare bool, pullQuery bool) (planner.PhysicalPlan, planner.LogicalPlan, error) { - ast, err := p.Parse(query) +func (p *Planner) QueryToPlan(query string, prepare bool, pullQuery bool) (planner.PhysicalPlan, planner.LogicalPlan, int, error) { + phys, log, paramCount, err := p.doQueryToPlan(query, prepare, pullQuery) if err != nil { - return nil, nil, errors.WithStack(err) + return nil, nil, 0, errors.MaybeConvertToPranaErrorf(err, errors.InvalidStatement, err.Error()) + } + return phys, log, paramCount, nil +} + +func (p *Planner) doQueryToPlan(query string, prepare bool, pullQuery bool) (planner.PhysicalPlan, planner.LogicalPlan, int, error) { + ast, paramCount, err := p.Parse(query) + if err != nil { + return nil, nil, 0, err } logic, err := p.BuildLogicalPlan(ast, prepare) if err != nil { - return nil, nil, errors.WithStack(err) + return nil, nil, 0, err } physical, err := p.BuildPhysicalPlan(logic, pullQuery) if err != nil { - return nil, nil, errors.WithStack(err) + return nil, nil, 0, err } - return physical, logic, nil + return physical, logic, paramCount, nil } func (p *Planner) BuildLogicalPlan(stmt AstHandle, prepare bool) (planner.LogicalPlan, error) { diff --git a/pull/engine.go b/pull/engine.go index 6e34e42a..87a00e48 100644 --- a/pull/engine.go +++ b/pull/engine.go @@ -2,6 +2,7 @@ package pull import ( "fmt" + log "github.com/sirupsen/logrus" "github.com/squareup/pranadb/sharder" "strings" "sync" @@ -69,7 +70,7 @@ func (p *Engine) BuildPullQuery(execCtx *execctx.ExecutionContext, query string, qi.ExecutionID = execCtx.ID qi.SchemaName = execCtx.Schema.Name qi.Query = query - isPs := argTypes != nil + isPs := len(argTypes) != 0 if isPs { // It's a prepared statement execCtx.Planner().SetPSArgs(args) @@ -77,13 +78,18 @@ func (p *Engine) BuildPullQuery(execCtx *execctx.ExecutionContext, query string, qi.PsArgs = args qi.PreparedStatement = true } - ast, err := execCtx.Planner().Parse(query) + ast, paramCount, err := execCtx.Planner().Parse(query) if err != nil { - return nil, errors.WithStack(err) + log.Errorf("failed to parse query %v", err) + return nil, errors.MaybeConvertToPranaErrorf(err, errors.InvalidStatement, "Invalid statement %s - %s", query, err.Error()) + } + if paramCount != len(argTypes) { + return nil, errors.NewPranaErrorf(errors.InvalidParamCount, "Statement has %d param markers but %d param(s) supplied", + paramCount, len(argTypes)) } logicalPlan, err := execCtx.Planner().BuildLogicalPlan(ast, isPs) if err != nil { - return nil, errors.WithStack(err) + return nil, errors.MaybeConvertToPranaErrorf(err, errors.InvalidStatement, "Invalid statement %s - %s", query, err.Error()) } physicalPlan, err := execCtx.Planner().BuildPhysicalPlan(logicalPlan, true) if err != nil { @@ -112,7 +118,7 @@ func (p *Engine) ExecuteRemotePullQuery(queryInfo *cluster.QueryExecutionInfo) ( execCtx = execctx.NewExecutionContext(queryInfo.ExecutionID, schema) newExecution = true execCtx.QueryInfo = queryInfo - ast, err := execCtx.Planner().Parse(queryInfo.Query) + ast, _, err := execCtx.Planner().Parse(queryInfo.Query) if err != nil { return nil, errors.WithStack(err) } diff --git a/pull/exec/limit.go b/pull/exec/limit.go index 9b38ecd0..ab2875de 100644 --- a/pull/exec/limit.go +++ b/pull/exec/limit.go @@ -32,17 +32,17 @@ func NewPullLimit(colNames []string, colTypes []common.ColumnType, count, offset func (l *PullLimit) GetRows(maxRowsToReturn int) (*common.Rows, error) { if maxRowsToReturn < 1 { - return nil, errors.Errorf("invalid limit %d", maxRowsToReturn) + return nil, errors.Errorf("Invalid limit %d", maxRowsToReturn) } // OFFSET is unsupported for now. if l.offset != 0 { - return nil, errors.NewInvalidStatementError("offset must be zero") + return nil, errors.NewInvalidStatementError("Offset must be zero") } // Because LIMIT is often used together with ORDER BY which is limited to orderByMaxRows rows, // we impose the same max on LIMIT. if l.count > orderByMaxRows { return nil, errors.NewInvalidStatementError( - fmt.Sprintf("limit count cannot be larger than %d", orderByMaxRows), + fmt.Sprintf("Limit count cannot be larger than %d", orderByMaxRows), ) } if l.count == 0 { diff --git a/pull/exec_builder.go b/pull/exec_builder.go index ff71659c..bdd091a9 100644 --- a/pull/exec_builder.go +++ b/pull/exec_builder.go @@ -2,6 +2,7 @@ package pull import ( "fmt" + log "github.com/sirupsen/logrus" "github.com/squareup/pranadb/tidb/sessionctx" "strings" @@ -115,7 +116,8 @@ func (p *Engine) buildPullDAG(ctx *execctx.ExecutionContext, plan planner.Physic sort := exec.NewPullSort(colNames, colTypes, desc, sortByExprs) executor = exec.NewPullChain(limit, sort) default: - return nil, errors.Errorf("unexpected plan type %T", plan) + log.Errorf("unexpected plan type %T", plan) + return nil, errors.NewPranaErrorf(errors.InvalidStatement, ctx.QueryInfo.Query) } var childExecutors []exec.PullExecutor diff --git a/push/engine.go b/push/engine.go index 6c418a67..5c909fd8 100644 --- a/push/engine.go +++ b/push/engine.go @@ -272,7 +272,7 @@ func (p *Engine) getTableExecutorForIndex(indexInfo *common.IndexInfo) (*exec.Ta if !ok { mvInfo, ok := p.meta.GetMaterializedView(indexInfo.SchemaName, indexInfo.TableName) if !ok { - return nil, errors.NewUnknownSourceOrMaterializedViewError(indexInfo.SchemaName, indexInfo.TableName) + return nil, errors.NewUnknownTableError(indexInfo.SchemaName, indexInfo.TableName) } mv, err := p.GetMaterializedView(mvInfo.ID) if err != nil { @@ -613,7 +613,7 @@ func (p *Engine) CreateSource(sourceInfo *common.SourceInfo) (*source.Source, er }() pl := parplan.NewPlanner(schema) query := fmt.Sprintf("select * from %s where %s", tabName, ingestFilter) - phys, _, err := pl.QueryToPlan(query, false, false) + phys, _, _, err := pl.QueryToPlan(query, false, false) var sel *planner.PhysicalSelection if err == nil { var ok bool @@ -624,7 +624,7 @@ func (p *Engine) CreateSource(sourceInfo *common.SourceInfo) (*source.Source, er } } if err != nil || sel == nil { - return nil, errors.NewPranaErrorf(errors.InvalidIngestFilter, "invalid ingest filter \"%s\"", ingestFilter) + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "invalid ingest filter \"%s\"", ingestFilter) } ingestExpressions = make([]*common.Expression, len(sel.Conditions)) for i, expr := range sel.Conditions { diff --git a/push/exec_builder.go b/push/exec_builder.go index e4defc5f..ce072591 100644 --- a/push/exec_builder.go +++ b/push/exec_builder.go @@ -18,7 +18,7 @@ func (m *MaterializedView) buildPushQueryExecution(pl *parplan.Planner, schema * seqGenerator common.SeqGenerator) (exec.PushExecutor, []*common.InternalTableInfo, error) { // Build the physical plan - physicalPlan, logicalPlan, err := pl.QueryToPlan(query, false, false) + physicalPlan, logicalPlan, _, err := pl.QueryToPlan(query, false, false) if err != nil { return nil, nil, errors.WithStack(err) } @@ -85,7 +85,7 @@ func (m *MaterializedView) buildPushDAG(plan planner.PhysicalPlan, aggSequence i funcType = aggfuncs.FirstRowAggregateFunctionType firstRowFuncs++ default: - return nil, nil, errors.Errorf("unexpected aggregate function %s", aggFunc.Name) + return nil, nil, errors.NewPranaErrorf(errors.InvalidStatement, "Unsupported aggregate function %s", aggFunc.Name) } colType := common.ConvertTiDBTypeToPranaType(aggFunc.RetTp) af := &exec.AggregateFunctionInfo{ diff --git a/push/source/source.go b/push/source/source.go index acce45e4..b13734ab 100644 --- a/push/source/source.go +++ b/push/source/source.go @@ -98,16 +98,13 @@ func NewSource(sourceInfo *common.SourceInfo, tableExec *exec.TableExecutor, ing // TODO we should validate the sourceinfo - e.g. check that number of col selectors, column names and column types are the same var msgProvFact kafka.MessageProviderFactory ti := sourceInfo.TopicInfo - if ti == nil { - // TODO not sure if we need this... parser should catch it? - return nil, errors.NewPranaErrorf(errors.MissingTopicInfo, "No topic info configured for source %s", sourceInfo.Name) + var brokerConf conf.BrokerConfig + var ok bool + if cfg.KafkaBrokers != nil { + brokerConf, ok = cfg.KafkaBrokers[ti.BrokerName] } - if cfg.KafkaBrokers == nil { - return nil, errors.NewPranaError(errors.MissingKafkaBrokers, "No Kafka brokers configured") - } - brokerConf, ok := cfg.KafkaBrokers[ti.BrokerName] - if !ok { - return nil, errors.NewPranaErrorf(errors.UnknownBrokerName, "Unknown broker. Name: %s", ti.BrokerName) + if !ok || cfg.KafkaBrokers == nil { + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Unknown broker %s - has it been configured in the server config?", ti.BrokerName) } props := copyAndAddAll(brokerConf.Properties, ti.Properties) groupID := GenerateGroupID(cfg.ClusterID, sourceInfo) @@ -121,7 +118,7 @@ func NewSource(sourceInfo *common.SourceInfo, tableExec *exec.TableExecutor, ing case conf.BrokerClientDefault: msgProvFact = kafka.NewMessageProviderFactory(ti.TopicName, props, groupID) default: - return nil, errors.NewPranaErrorf(errors.UnsupportedBrokerClientType, "Unsupported broker client type %d", brokerConf.ClientType) + return nil, errors.NewPranaErrorf(errors.InvalidStatement, "Unsupported broker client type %d", brokerConf.ClientType) } numConsumers, err := getOrDefaultIntValue(numConsumersPerSourcePropName, sourceInfo.TopicInfo.Properties, defaultNumConsumersPerSource) if err != nil { diff --git a/remoting/remoting_test.go b/remoting/remoting_test.go index 7c912f3b..98997250 100644 --- a/remoting/remoting_test.go +++ b/remoting/remoting_test.go @@ -290,7 +290,7 @@ func TestSyncBroadcastWithFailingNotif(t *testing.T) { require.Error(t, err) perr, ok := err.(errors.PranaError) require.True(t, ok) - require.True(t, strings.HasPrefix(perr.Error(), "PDB0000 - Internal error - reference:")) + require.True(t, strings.HasPrefix(perr.Error(), "PDB5000 - Internal error - reference:")) require.Equal(t, errors.InternalError, int(perr.Code)) listeners[1].SetReturnError(errors.NewPranaError(errors.InvalidStatement, "avocados")) @@ -306,14 +306,14 @@ func TestSyncBroadcastWithFailingNotif(t *testing.T) { require.NoError(t, err) for i := 0; i < numServers; i++ { - listeners[i].SetReturnError(errors.NewPranaError(errors.InvalidIngestFilter, "tomatoes")) + listeners[i].SetReturnError(errors.NewPranaError(errors.InvalidStatement, "tomatoes")) } err = client.BroadcastSync(notif) require.Error(t, err) perr, ok = err.(errors.PranaError) require.True(t, ok) require.Equal(t, "tomatoes", perr.Error()) - require.Equal(t, errors.InvalidIngestFilter, int(perr.Code)) + require.Equal(t, errors.InvalidStatement, int(perr.Code)) } func TestSendRequest(t *testing.T) { @@ -478,7 +478,7 @@ func TestSendRequestWithError(t *testing.T) { defer stopServers(t, server) server.RegisterMessageHandler(ClusterMessageClusterProposeRequest, nListener) - nListener.SetReturnError(errors.NewPranaError(errors.UnknownIndexColumn, "some request error")) + nListener.SetReturnError(errors.NewPranaError(errors.InvalidStatement, "some request error")) err := server.Start() require.NoError(t, err) @@ -499,7 +499,7 @@ func TestSendRequestWithError(t *testing.T) { perr, ok := err.(errors.PranaError) require.True(t, ok) require.Equal(t, "some request error", perr.Error()) - require.Equal(t, errors.UnknownIndexColumn, int(perr.Code)) + require.Equal(t, errors.InvalidStatement, int(perr.Code)) } func waitForNotifications(t *testing.T, notifListeners []*notifListener, numNotificatiuons int) { diff --git a/sqltest/sql_test_runner.go b/sqltest/sql_test_runner.go index 54323b0e..177cc112 100644 --- a/sqltest/sql_test_runner.go +++ b/sqltest/sql_test_runner.go @@ -555,42 +555,45 @@ func (st *sqlTest) runTestIteration(require *require.Assertions, commands []stri require.NoError(err) expectedLines := strings.Split(string(b), "\n") actualLines := strings.Split(st.output.String(), "\n") - require.Equal(len(expectedLines), len(actualLines), "expected and actual output number of lines not equal. expected %d actual %d", - len(expectedLines), len(actualLines)) - // We compare the lines one by one because there are special lines that we compare by prefix not exactly - // E.g. internal error contains a UUID which is different each time so we can't exact compare ok := true - for i, expected := range expectedLines { - actual := actualLines[i] - hasPrefix := false - for _, prefix := range prefixCompareLines { - if strings.HasPrefix(expected, prefix) { - if !strings.HasPrefix(actual, prefix) { - ok = false + if len(expectedLines) != len(actualLines) { + ok = false + } else { + // We compare the lines one by one because there are special lines that we compare by prefix not exactly + // E.g. internal error contains a UUID which is different each time so we can't exact compare + for i, expected := range expectedLines { + actual := actualLines[i] + hasPrefix := false + for _, prefix := range prefixCompareLines { + if strings.HasPrefix(expected, prefix) { + if !strings.HasPrefix(actual, prefix) { + ok = false + } + hasPrefix = true + break } - hasPrefix = true + } + if !ok { + break + } + a := strings.TrimRight(actual, " ") + e := strings.TrimRight(expected, " ") + if !hasPrefix && a != e { + ok = false break } - } - if !ok { - break - } - if !hasPrefix && actual != expected { - ok = false - break } } if !ok { require.Equal(string(b), st.output.String()) } } - dur := time.Now().Sub(start) log.Infof("Finished running sql test %s time taken %d ms", st.testName, dur.Milliseconds()) return numIters } -var prefixCompareLines = []string{"Failed to execute statement: PDB0000 - Internal error - reference:"} +var prefixCompareLines = []string{"Failed to execute statement: PDB5000 - Internal error - reference:"} func (st *sqlTest) waitUntilRowsInTable(require *require.Assertions, tableName string, numRows int) { lineExpected := fmt.Sprintf("%d rows returned", numRows) diff --git a/sqltest/testdata/basic_mv_test_out.txt b/sqltest/testdata/basic_mv_test_out.txt index 0329d377..aa747ac4 100644 --- a/sqltest/testdata/basic_mv_test_out.txt +++ b/sqltest/testdata/basic_mv_test_out.txt @@ -62,17 +62,17 @@ select * from test_mv_1 order by col0; -- try and drop the source before the MV is dropped - this should fail; drop source test_source_1; -Failed to execute statement: PDB0010 - Cannot drop source test.test_source_1 it has the following children test.test_mv_1 +Failed to execute statement: PDB1009 - Cannot drop source test.test_source_1 it has the following children test.test_mv_1 drop materialized view test_mv_1; 0 rows returned select * from test_mv_1 order by col0; -Failed to execute statement: PDB0002 - Table 'test.test_mv_1' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_mv_1' doesn't exist -- try and drop it again; drop materialized view test_mv_1; -Failed to execute statement: PDB0006 - Unknown materialized view: test.test_mv_1 +Failed to execute statement: PDB1003 - Unknown materialized view: test.test_mv_1 drop source test_source_1; 0 rows returned diff --git a/sqltest/testdata/basic_source_test_out.txt b/sqltest/testdata/basic_source_test_out.txt index f4d55a49..fe50f698 100644 --- a/sqltest/testdata/basic_source_test_out.txt +++ b/sqltest/testdata/basic_source_test_out.txt @@ -61,9 +61,9 @@ drop source test_source_1; 0 rows returned select * from test_source_1 order by col0; -Failed to execute statement: PDB0002 - Table 'test.test_source_1' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_source_1' doesn't exist drop source test_source_1; -Failed to execute statement: PDB0005 - Unknown source: test.test_source_1 +Failed to execute statement: PDB1002 - Unknown source: test.test_source_1 --delete topic testtopic; diff --git a/sqltest/testdata/cascading_mvs_test_out.txt b/sqltest/testdata/cascading_mvs_test_out.txt index 6c052e75..5a4c0ac3 100644 --- a/sqltest/testdata/cascading_mvs_test_out.txt +++ b/sqltest/testdata/cascading_mvs_test_out.txt @@ -211,15 +211,15 @@ select * from test_mv_6 order by col0; --now try and drop mvs when they stil have children - this should fail; drop materialized view test_mv_5; -Failed to execute statement: PDB0011 - Cannot drop materialized view test.test_mv_5 it has the following children test.test_mv_6 +Failed to execute statement: PDB1010 - Cannot drop materialized view test.test_mv_5 it has the following children test.test_mv_6 drop materialized view test_mv_4; -Failed to execute statement: PDB0011 - Cannot drop materialized view test.test_mv_4 it has the following children test.test_mv_5 +Failed to execute statement: PDB1010 - Cannot drop materialized view test.test_mv_4 it has the following children test.test_mv_5 drop materialized view test_mv_3; -Failed to execute statement: PDB0011 - Cannot drop materialized view test.test_mv_3 it has the following children test.test_mv_4 +Failed to execute statement: PDB1010 - Cannot drop materialized view test.test_mv_3 it has the following children test.test_mv_4 drop materialized view test_mv_2; -Failed to execute statement: PDB0011 - Cannot drop materialized view test.test_mv_2 it has the following children test.test_mv_3 +Failed to execute statement: PDB1010 - Cannot drop materialized view test.test_mv_2 it has the following children test.test_mv_3 drop materialized view test_mv_1; -Failed to execute statement: PDB0011 - Cannot drop materialized view test.test_mv_1 it has the following children test.test_mv_2 +Failed to execute statement: PDB1010 - Cannot drop materialized view test.test_mv_1 it has the following children test.test_mv_2 --now restart - they should be recreated properly on restart; --restart cluster; diff --git a/sqltest/testdata/compound_pk_test_out.txt b/sqltest/testdata/compound_pk_test_out.txt index fec3cd04..293486cf 100644 --- a/sqltest/testdata/compound_pk_test_out.txt +++ b/sqltest/testdata/compound_pk_test_out.txt @@ -54,9 +54,9 @@ drop source test_source_1; 0 rows returned select * from test_source_1 order by col0; -Failed to execute statement: PDB0002 - Table 'test.test_source_1' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_source_1' doesn't exist drop source test_source_1; -Failed to execute statement: PDB0005 - Unknown source: test.test_source_1 +Failed to execute statement: PDB1002 - Unknown source: test.test_source_1 --delete topic testtopic; diff --git a/sqltest/testdata/describe_test_out.txt b/sqltest/testdata/describe_test_out.txt index 3cfe92c7..c2a8d2aa 100644 --- a/sqltest/testdata/describe_test_out.txt +++ b/sqltest/testdata/describe_test_out.txt @@ -2,7 +2,7 @@ use test; 0 rows returned describe foo; -Failed to execute statement: PDB0019 - Unknown source or materialized view: test.foo +Failed to execute statement: PDB1005 - Unknown source or materialized view: test.foo --create topic testtopic; @@ -65,11 +65,11 @@ describe test_mv; drop materialized view test_mv; 0 rows returned describe test_mv; -Failed to execute statement: PDB0019 - Unknown source or materialized view: test.test_mv +Failed to execute statement: PDB1005 - Unknown source or materialized view: test.test_mv drop source test_source; 0 rows returned describe test_source; -Failed to execute statement: PDB0019 - Unknown source or materialized view: test.test_source +Failed to execute statement: PDB1005 - Unknown source or materialized view: test.test_source --delete topic testtopic; diff --git a/sqltest/testdata/errors_test_data.txt b/sqltest/testdata/errors_test_data.txt new file mode 100644 index 00000000..bffc0c3d --- /dev/null +++ b/sqltest/testdata/errors_test_data.txt @@ -0,0 +1,11 @@ +dataset:dataset_1 test_source_1 +1,10,1000,1234.4321,12345678.99,str1,2020-01-01 01:00:00.123456 +2,20,2000,2234.4321,22345678.99,str2,2020-01-02 01:00:00.123456 +3,30,3000,3234.4321,32345678.99,str3,2020-01-03 01:00:00.123456 +4,40,4000,4234.4321,42345678.99,str4,2020-01-04 01:00:00.123456 +5,50,5000,5234.4321,52345678.99,str5,2020-01-05 01:00:00.123456 +6,60,6000,6234.4321,62345678.99,str6,2020-01-06 01:00:00.123456 +7,70,7000,7234.4321,72345678.99,str7,2020-01-07 01:00:00.123456 +8,80,8000,8234.4321,82345678.99,str8,2020-01-08 01:00:00.123456 +9,90,9000,9234.4321,92345678.99,str9,2020-01-09 01:00:00.123456 +10,100,10000,10234.4321,93345678.99,str10,2020-01-10 01:00:00.123456 \ No newline at end of file diff --git a/sqltest/testdata/errors_test_out.txt b/sqltest/testdata/errors_test_out.txt new file mode 100644 index 00000000..f9fe09e8 --- /dev/null +++ b/sqltest/testdata/errors_test_out.txt @@ -0,0 +1,1102 @@ +-- client errors; +----------------; + +-- set errors; + +set; +Failed to execute statement: Invalid set command. Should be set +set whateva; +Failed to execute statement: Invalid set command. Should be set +set foobar blah; +Failed to execute statement: Unknown property: foobar +set max_line_width; +Failed to execute statement: Invalid set command. Should be set +set max_line_width 10000000; +Failed to execute statement: Invalid max_line_width value: 10000000 +set max_line_width 0; +Failed to execute statement: Invalid max_line_width value: 0 +set max_line_width -1; +Failed to execute statement: Invalid max_line_width value: -1 +set max_line_width -100; +Failed to execute statement: Invalid max_line_width value: -100 +set max_line_width bananas; +Failed to execute statement: Invalid max_line_width value: bananas + +-- general syntax errors; + +uiuqwdiuqwdiuqwhd; +Failed to execute statement: PDB1000 - 1:1: unexpected token "uiuqwdiuqwdiuqwhd" +ijwsoiqwd qwudhquwidh; +Failed to execute statement: PDB1000 - 1:1: unexpected token "ijwsoiqwd" + +iwdoqwd +qwdqwdqw +qwdqwdqwd +qwdqwd; +Failed to execute statement: PDB1000 - 1:1: unexpected token "iwdoqwd" + +7; +Failed to execute statement: PDB1000 - 1:1: unexpected token "7" + +create sausages; +Failed to execute statement: PDB1000 - 1:8: unexpected token "sausages" (expected Create) +cr eate source; +Failed to execute statement: PDB1000 - 1:1: unexpected token "cr" + +-- create source errors; + +--create topic testtopic; + +create source foo( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1001 - No schema in use + +use test; +0 rows returned + +create source foo( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +0 rows returned + +-- already exists; +create source foo( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1006 - Source test.foo already exists + +drop source foo; +0 rows returned + +create source 34353( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 1:15: unexpected token "34353" (expected CreateSource) + +create source !*£8373( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 1:15: invalid input text "!*£8373(\n col..." + +create source ( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 1:15: unexpected token "(" (expected CreateSource) + +create source bar( + 23123 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: unexpected token "23123" (expected TableOption ("," TableOption)* ")" "WITH" "(" TopicInformation ("," TopicInformation)* ")") + +create source bar( + col0 ginormousint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:10: unexpected token "ginormousint" (expected ("VARCHAR" | "TINYINT" | "INT" | "BIGINT" | "TIMESTAMP" | "DOUBLE" | "DECIMAL") ("(" ("," )* ")")?) + +create source bar( + col0 decimal(0,0), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: Decimal precision must be > 0 and <= 65 + +create source bar( + col0 decimal(-1,0), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: Decimal precision must be > 0 and <= 65 + +create source bar( + col0 decimal(66,0), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: Decimal precision must be > 0 and <= 65 + +create source bar( + col0 decimal(65,-1), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: decimal scale must be >= 0 and <= 30 + +create source bar( + col0 decimal(65,31), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: decimal scale must be >= 0 and <= 30 + +create source bar( + col0 decimal(29,30), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: Decimal scale must be <= precision + +create source bar( + col0 decimal, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: Expected DECIMAL(precision, scale) + +create source bar( + col0 decimal(), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:17: unexpected token "(" (expected ")" "WITH" "(" TopicInformation ("," TopicInformation)* ")") + +create source bar( + col0 decimal(45), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: Expected DECIMAL(precision, scale) + +create source bar( + col0 timestamp(-1), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: Timestamp fsp must be >= 0 and <= 6 + +create source bar( + col0 timestamp(7), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 2:5: Timestamp fsp must be >= 0 and <= 6 + +create source bar( + col0 bigint + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 3:5: unexpected token "primary" (expected ")" "WITH" "(" TopicInformation ("," TopicInformation)* ")") + +create source bar( + col0 bigint, + primary key (col0), +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 3:23: unexpected token "," (expected ")" "WITH" "(" TopicInformation ("," TopicInformation)* ")") + +create source bar( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic" + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - 7:5: unexpected token "headerencoding" (expected ")") + +create source bar( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ), +); +Failed to execute statement: PDB1000 - 12:6: unexpected token "," (expected ")") + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v1, + ) +); +Failed to execute statement: PDB1000 - 13:11: unexpected token "," (expected ")") + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v1 + ); +Failed to execute statement: PDB1000 - 14:6: unexpected token "" (expected ")") + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + broke rname = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v1 + ) +); +Failed to execute statement: PDB1000 - 6:5: unexpected token "broke" (expected TopicInformation ("," TopicInformation)* ")") + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); +Failed to execute statement: PDB1000 - Number of column selectors (1) must match number of columns (2) + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "george", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - Unknown broker george - has it been configured in the server config? + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "foo", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - Unknown topic encoding foo + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "foo", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - Unknown topic encoding foo + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "foo", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - Unknown topic encoding foo + +create source bar( + col0 bigint, + col1 bigint +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - Primary key is required + +create source bar( + col0 bigint, + col1 bigint, + primary key () +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - 4:18: unexpected token ")" (expected ("," )* ")") + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0, col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - Primary key cannot contain same column multiple times + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0, col1, col2) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - Invalid primary key column ["col0" "col1" "col2"] + +create source bar( + col0 bigint, + col1 bigint, + primary key (col2) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - Invalid primary key column ["col2"] + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - brokerName is required + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - topicName is required + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - headerEncoding is required + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - keyEncoding is required + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1000 - valueEncoding is required + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json" +); +Failed to execute statement: PDB1000 - Number of column selectors (0) must match number of columns (2) + +--errors in drop source; + +drop source who; +Failed to execute statement: PDB1002 - Unknown source: test.who +drop source 1254124; +Failed to execute statement: PDB1000 - 1:13: unexpected token "1254124" (expected ("ON" )?) +drop source; +Failed to execute statement: PDB1000 - 1:12: unexpected token "" (expected ("ON" )?) +drop source uqwhs qwdiuhqwd; +Failed to execute statement: PDB1000 - 1:19: unexpected token "qwdiuhqwd" + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +0 rows returned + +create materialized view mv1 as select * from bar; +0 rows returned + +drop source bar; +Failed to execute statement: PDB1009 - Cannot drop source test.bar it has the following children test.mv1 + +drop materialized view mv1; +0 rows returned +drop source bar; +0 rows returned + +-- show and describe errors; + +show sch emas; +Failed to execute statement: PDB1000 - 1:6: unexpected token "sch" (expected Show) +show schemass; +Failed to execute statement: PDB1000 - 1:6: unexpected token "schemass" (expected Show) +show; +Failed to execute statement: PDB1000 - 1:5: unexpected token "" (expected Show) +show t ables; +Failed to execute statement: PDB1000 - 1:6: unexpected token "t" (expected Show) +show ttables; +Failed to execute statement: PDB1000 - 1:6: unexpected token "ttables" (expected Show) + +describe a b; +Failed to execute statement: PDB1000 - 1:12: unexpected token "b" +describe blah; +Failed to execute statement: PDB1005 - Unknown source or materialized view: test.blah + +-- create materialized view errors; + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +0 rows returned + +create materialized view bar as select * from bar; +Failed to execute statement: PDB1006 - Source test.bar already exists + +create materialized view mv1 as select * from bar; +0 rows returned + +create materialized view mv1 as select * from bar; +Failed to execute statement: PDB1007 - Materialized view test.mv1 already exists + +create source mv1( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +Failed to execute statement: PDB1007 - Materialized view test.mv1 already exists + +drop materialized view mv1; +0 rows returned +drop source bar; +0 rows returned + +-- drop materialized view errors; + +drop materialized view who; +Failed to execute statement: PDB1003 - Unknown materialized view: test.who +drop materialized view 1254124; +Failed to execute statement: PDB1000 - 1:24: unexpected token "1254124" (expected ("ON" )?) +drop materialized view; +Failed to execute statement: PDB1000 - 1:23: unexpected token "" (expected ("ON" )?) +drop materialized view uqwhs qwdiuhqwd; +Failed to execute statement: PDB1000 - 1:30: unexpected token "qwdiuhqwd" + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +0 rows returned + +create materialized view mv1 as select * from bar; +0 rows returned + +create materialized view mv2 as select * from mv1; +0 rows returned + +drop materialized view mv1; +Failed to execute statement: PDB1010 - Cannot drop materialized view test.mv1 it has the following children test.mv2 + +drop materialized view mv2; +0 rows returned +drop materialized view mv1; +0 rows returned +drop source bar; +0 rows returned + +-- create index errors; + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +0 rows returned + +create index 51424 on bar(col1); +Failed to execute statement: PDB1000 - 1:14: unexpected token "51424" (expected CreateIndex) + +create index on bar(col1); +Failed to execute statement: PDB1000 - 1:17: unexpected token "bar" (expected "ON" "(" ColumnName ("," ColumnName)* ")") + +create index foo(col1); +Failed to execute statement: PDB1000 - 1:17: unexpected token "(" (expected "ON" "(" ColumnName ("," ColumnName)* ")") + +create index foo on bar; +Failed to execute statement: PDB1000 - 1:24: unexpected token "" (expected "(" ColumnName ("," ColumnName)* ")") + +create index foo on bar(); +Failed to execute statement: PDB1000 - 1:25: unexpected token ")" (expected ColumnName ("," ColumnName)* ")") + +create index foo on bar(col2); +Failed to execute statement: PDB1000 - Unknown column col2 in test.bar + +create index foo on bar(col1, col1); +Failed to execute statement: PDB1000 - Index cannot contain same column multiple times + +create index foo on bar(col1, col2); +Failed to execute statement: PDB1000 - Unknown column col2 in test.bar + +create index foo on bar(col1); +0 rows returned +create index foo on bar(col1); +Failed to execute statement: PDB1008 - Index foo already exists on test.bar + +drop index foo on bar; +0 rows returned +drop source bar; +0 rows returned + +-- drop index errors; + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); +0 rows returned + +drop index foo on bar; +Failed to execute statement: PDB1004 - Unknown index: test.bar.foo + +create index foo on bar(col1); +0 rows returned +drop index foo; +Failed to execute statement: PDB1000 - Drop index requires a table + +drop index foo on bar; +0 rows returned +drop source bar; +0 rows returned + +-- query errors; + +create source bar( + col0 bigint, + col1 tinyint, + col2 int, + col3 double, + col4 decimal(10, 2), + col5 varchar, + col6 timestamp, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + v0, + v1, + v2, + v3, + v4, + v5, + v6 + ) +); +0 rows returned + +se lect * from bar; +Failed to execute statement: PDB1000 - 1:1: unexpected token "se" + +select * from unknown; +Failed to execute statement: PDB1000 - Table 'test.unknown' doesn't exist + +select col100 from bar; +Failed to execute statement: PDB1000 - Unknown column 'col100' in 'field list' + +select col0, col100 from bar; +Failed to execute statement: PDB1000 - Unknown column 'col100' in 'field list' + +create materialized view foo as select * from unknown; +Failed to execute statement: PDB1000 - Table 'test.unknown' doesn't exist + +create materialized view foo as select col100 from bar; +Failed to execute statement: PDB1000 - Unknown column 'col100' in 'field list' + +create materialized view foo as select col0, col100 from bar; +Failed to execute statement: PDB1000 - Unknown column 'col100' in 'field list' + +select foofunc(col0) from bar; +Failed to execute statement: PDB1000 - FUNCTION test.foofunc does not exist + +create materialized view foo as select foofunc(col0) from bar; +Failed to execute statement: PDB1000 - FUNCTION test.foofunc does not exist + +select lpad(col5, 100, "abc", 23, 67) from bar; +Failed to execute statement: PDB1000 - Incorrect parameter count in the call to native function 'lpad' + +-- we don't currently support max min or avg; +create materialized view foo as select col5, max(col1) from bar group by col5; +Failed to execute statement: PDB1000 - Unsupported aggregate function max +create materialized view foo as select col5, min(col1) from bar group by col5; +Failed to execute statement: PDB1000 - Unsupported aggregate function min +create materialized view foo as select col5, avg(col1) from bar group by col5; +Failed to execute statement: PDB1000 - Unsupported aggregate function avg + +create materialized view foo as select col5, unknown(col1) from bar group by col5; +Failed to execute statement: PDB1000 - line 1 column 22 near "(col1) from bar group by col5" + +create materialized view foo as select col5, sum(col1) from bar group by col5 having col7 > 10; +Failed to execute statement: PDB1000 - Unknown column 'col7' in 'having clause' + +-- we don't support joins in queries; + +create source quux( + col0 bigint, + col1 tinyint, + col2 int, + col3 double, + col4 decimal(10, 2), + col5 varchar, + col6 timestamp, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + v0, + v1, + v2, + v3, + v4, + v5, + v6 + ) +); +0 rows returned + +create materialized view foo as select a.col0, a.col5, b.col2 from bar a inner join quux b on a.col0 = b.col0; +Failed to execute statement: PDB1000 - Internal : Can't find a proper physical plan for this query + +select a.col0, a.col5, b.col2 from bar a inner join quux b on a.col0 = b.col0; +Failed to execute statement: PDB1000 - Internal : Can't find a proper physical plan for this query + +-- pull queries don't support aggregations; + +select col5, sum(col1) from bar group by col5; +Failed to execute statement: PDB1000 - select col5, sum(col1) from bar group by col5 + +-- we don't support sub queries; + +select col0, col3 from bar where exists (select 1 from quux where quux.col0 = bar.col0); +Failed to execute statement: PDB1000 - Invalid statement select col0, col3 from bar where exists (select 1 from quux where quux.col0 = bar.col0) - UnknownType: *ast.SelectField + +select col0, col3 from bar where col3 > (select sum(col1) from quux); +Failed to execute statement: PDB1000 - Invalid statement select col0, col3 from bar where col3 > (select sum(col1) from quux) - Invalid use of group function + +create materialized view foo as select col0, col3 from bar where exists (select 1 from quux where quux.col0 = bar.col0); +Failed to execute statement: PDB1000 - UnknownType: *ast.SelectField + +create materialized view foo as select col0, col3 from bar where col3 > (select sum(col1) from quux); +Failed to execute statement: PDB1000 - Invalid use of group function + +drop source quux; +0 rows returned + +-- prepared statements errors; + +select * from bar where col0 = ?; +Failed to execute statement: PDB1011 - Statement has 1 param markers but 0 param(s) supplied + +execps 0 "select * from bar where col0 = ?"; +Failed to execute statement: PDB1011 - Statement has 1 param markers but 0 param(s) supplied + +execps 1 bigint 23 "select * from bar where col0 = ? and col1 = ?"; +Failed to execute statement: PDB1011 - Statement has 2 param markers but 1 param(s) supplied + +create materialized view foo as select col0 from bar where col0 = ?; +Failed to execute statement: PDB1000 - 1:67: invalid input text "?" + +execps 1 bigint 23 "select * from bar where col0 = ??"; +Failed to execute statement: PDB1000 - Invalid statement select * from bar where col0 = ?? - line 1 column 33 near "?" + +execps 1 bigint 23 "select * from bar where col0 = $"; +Failed to execute statement: PDB1011 - Statement has 0 param markers but 1 param(s) supplied + +drop source bar; +0 rows returned + +--delete topic testtopic; diff --git a/sqltest/testdata/errors_test_script.txt b/sqltest/testdata/errors_test_script.txt new file mode 100644 index 00000000..6955aefe --- /dev/null +++ b/sqltest/testdata/errors_test_script.txt @@ -0,0 +1,958 @@ +-- client errors; +----------------; + +-- set errors; + +set; +set whateva; +set foobar blah; +set max_line_width; +set max_line_width 10000000; +set max_line_width 0; +set max_line_width -1; +set max_line_width -100; +set max_line_width bananas; + +-- general syntax errors; + +uiuqwdiuqwdiuqwhd; +ijwsoiqwd qwudhquwidh; + +iwdoqwd +qwdqwdqw +qwdqwdqwd +qwdqwd; + +7; + +create sausages; +cr eate source; + +-- create source errors; + +--create topic testtopic; + +create source foo( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +use test; + +create source foo( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +-- already exists; +create source foo( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +drop source foo; + +create source 34353( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source !*£8373( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source ( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + 23123 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 ginormousint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 decimal(0,0), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 decimal(-1,0), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 decimal(66,0), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 decimal(65,-1), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 decimal(65,31), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 decimal(29,30), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 decimal, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 decimal(), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 decimal(45), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 timestamp(-1), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 timestamp(7), + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 bigint + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 bigint, + primary key (col0), +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic" + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ), +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v1, + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v1 + ); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + broke rname = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v1 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "george", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "foo", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "foo", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "foo", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key () +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0, col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0, col1, col2) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col2) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json" +); + +--errors in drop source; + +drop source who; +drop source 1254124; +drop source; +drop source uqwhs qwdiuhqwd; + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create materialized view mv1 as select * from bar; + +drop source bar; + +drop materialized view mv1; +drop source bar; + +-- show and describe errors; + +show sch emas; +show schemass; +show; +show t ables; +show ttables; + +describe a b; +describe blah; + +-- create materialized view errors; + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create materialized view bar as select * from bar; + +create materialized view mv1 as select * from bar; + +create materialized view mv1 as select * from bar; + +create source mv1( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +drop materialized view mv1; +drop source bar; + +-- drop materialized view errors; + +drop materialized view who; +drop materialized view 1254124; +drop materialized view; +drop materialized view uqwhs qwdiuhqwd; + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create materialized view mv1 as select * from bar; + +create materialized view mv2 as select * from mv1; + +drop materialized view mv1; + +drop materialized view mv2; +drop materialized view mv1; +drop source bar; + +-- create index errors; + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +create index 51424 on bar(col1); + +create index on bar(col1); + +create index foo(col1); + +create index foo on bar; + +create index foo on bar(); + +create index foo on bar(col2); + +create index foo on bar(col1, col1); + +create index foo on bar(col1, col2); + +create index foo on bar(col1); +create index foo on bar(col1); + +drop index foo on bar; +drop source bar; + +-- drop index errors; + +create source bar( + col0 bigint, + col1 bigint, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + meta("key").k0, + v0 + ) +); + +drop index foo on bar; + +create index foo on bar(col1); +drop index foo; + +drop index foo on bar; +drop source bar; + +-- query errors; + +create source bar( + col0 bigint, + col1 tinyint, + col2 int, + col3 double, + col4 decimal(10, 2), + col5 varchar, + col6 timestamp, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + v0, + v1, + v2, + v3, + v4, + v5, + v6 + ) +); + +se lect * from bar; + +select * from unknown; + +select col100 from bar; + +select col0, col100 from bar; + +create materialized view foo as select * from unknown; + +create materialized view foo as select col100 from bar; + +create materialized view foo as select col0, col100 from bar; + +select foofunc(col0) from bar; + +create materialized view foo as select foofunc(col0) from bar; + +select lpad(col5, 100, "abc", 23, 67) from bar; + +-- we don't currently support max min or avg; +create materialized view foo as select col5, max(col1) from bar group by col5; +create materialized view foo as select col5, min(col1) from bar group by col5; +create materialized view foo as select col5, avg(col1) from bar group by col5; + +create materialized view foo as select col5, unknown(col1) from bar group by col5; + +create materialized view foo as select col5, sum(col1) from bar group by col5 having col7 > 10; + +-- we don't support joins in queries; + +create source quux( + col0 bigint, + col1 tinyint, + col2 int, + col3 double, + col4 decimal(10, 2), + col5 varchar, + col6 timestamp, + primary key (col0) +) with ( + brokername = "testbroker", + topicname = "testtopic", + headerencoding = "json", + keyencoding = "json", + valueencoding = "json", + columnselectors = ( + v0, + v1, + v2, + v3, + v4, + v5, + v6 + ) +); + +create materialized view foo as select a.col0, a.col5, b.col2 from bar a inner join quux b on a.col0 = b.col0; + +select a.col0, a.col5, b.col2 from bar a inner join quux b on a.col0 = b.col0; + +-- pull queries don't support aggregations; + +select col5, sum(col1) from bar group by col5; + +-- we don't support sub queries; + +select col0, col3 from bar where exists (select 1 from quux where quux.col0 = bar.col0); + +select col0, col3 from bar where col3 > (select sum(col1) from quux); + +create materialized view foo as select col0, col3 from bar where exists (select 1 from quux where quux.col0 = bar.col0); + +create materialized view foo as select col0, col3 from bar where col3 > (select sum(col1) from quux); + +drop source quux; + +-- prepared statements errors; + +select * from bar where col0 = ?; + +execps 0 "select * from bar where col0 = ?"; + +execps 1 bigint 23 "select * from bar where col0 = ? and col1 = ?"; + +create materialized view foo as select col0 from bar where col0 = ?; + +execps 1 bigint 23 "select * from bar where col0 = ??"; + +execps 1 bigint 23 "select * from bar where col0 = $"; + +drop source bar; + +--delete topic testtopic; \ No newline at end of file diff --git a/sqltest/testdata/index_test_out.txt b/sqltest/testdata/index_test_out.txt index 4e142754..0a22c3ed 100644 --- a/sqltest/testdata/index_test_out.txt +++ b/sqltest/testdata/index_test_out.txt @@ -47,9 +47,9 @@ create index index1 on test_mv_1 (col1); -- try and create index with same name on same tables - this should fail; create index index1 on test_source_1 (col1); -Failed to execute statement: PDB0021 - Index index1 already exists on test.test_source_1 +Failed to execute statement: PDB1008 - Index index1 already exists on test.test_source_1 create index index1 on test_mv_1 (col1); -Failed to execute statement: PDB0021 - Index index1 already exists on test.test_mv_1 +Failed to execute statement: PDB1008 - Index index1 already exists on test.test_mv_1 -- create a couple more indexes; create index index2 on test_source_1 (col1); @@ -69,9 +69,9 @@ drop index index2 on test_mv_1; -- try and drop some non existent indexes - should fail; drop index indexx on test_source_1; -Failed to execute statement: PDB0005 - Unknown index: test.test_source_1.indexx +Failed to execute statement: PDB1004 - Unknown index: test.test_source_1.indexx drop index indexx on test_mv_1; -Failed to execute statement: PDB0005 - Unknown index: test.test_mv_1.indexx +Failed to execute statement: PDB1004 - Unknown index: test.test_mv_1.indexx -- recreate indexes with same name; create index index1 on test_source_1 (col1); @@ -91,9 +91,9 @@ drop index index1 on test_mv_1; -- drop the source and mv - should fail as index2 needs to be explicitly deleted; drop materialized view test_mv_1; -Failed to execute statement: PDB0011 - Cannot drop materialized view test.test_mv_1 it has the following children test.test_mv_1.index2 +Failed to execute statement: PDB1010 - Cannot drop materialized view test.test_mv_1 it has the following children test.test_mv_1.index2 drop source test_source_1; -Failed to execute statement: PDB0010 - Cannot drop source test.test_source_1 it has the following children test.test_mv_1, test.test_source_1.index2 +Failed to execute statement: PDB1009 - Cannot drop source test.test_source_1 it has the following children test.test_mv_1, test.test_source_1.index2 drop index index2 on test_source_1; 0 rows returned diff --git a/sqltest/testdata/invalid_create_source_test_out.txt b/sqltest/testdata/invalid_create_source_test_out.txt index 5131073e..c3fd04eb 100644 --- a/sqltest/testdata/invalid_create_source_test_out.txt +++ b/sqltest/testdata/invalid_create_source_test_out.txt @@ -42,7 +42,7 @@ create source test_source_1( v1, ) ); -Failed to execute statement: PDB0002 - 13:11: unexpected token "," (expected ")") +Failed to execute statement: PDB1000 - 13:11: unexpected token "," (expected ")") -- TEST3 - invalid meta key on column selector; ------------------------------------------------------------; @@ -62,7 +62,7 @@ create source test_source_1( v1 ) ); -Failed to execute statement: PDB0018 - invalid metadata key in column selector "meta(\"notvalid\").k0". Valid values are "header", "key", "timestamp". +Failed to execute statement: PDB1000 - Invalid metadata key in column selector "meta(\"notvalid\").k0". Valid values are "header", "key", "timestamp". -- TEST4 - protobuf not registered; ------------------------------------------------------------; @@ -82,7 +82,6 @@ create source test_source_1( v1 ) ); -Failed to execute statement: PDB0016 - proto message "foo.bar.MissingType" not registered +Failed to execute statement: PDB1000 - Proto message "foo.bar.MissingType" not registered --delete topic testtopic; -; diff --git a/sqltest/testdata/invalid_create_source_test_script.txt b/sqltest/testdata/invalid_create_source_test_script.txt index 3e203be8..89a99360 100644 --- a/sqltest/testdata/invalid_create_source_test_script.txt +++ b/sqltest/testdata/invalid_create_source_test_script.txt @@ -78,4 +78,4 @@ create source test_source_1( ) ); ---delete topic testtopic; +--delete topic testtopic; \ No newline at end of file diff --git a/sqltest/testdata/multiple_mv_test_out.txt b/sqltest/testdata/multiple_mv_test_out.txt index 204e2978..ba0aa227 100644 --- a/sqltest/testdata/multiple_mv_test_out.txt +++ b/sqltest/testdata/multiple_mv_test_out.txt @@ -248,22 +248,22 @@ select * from test_mv_3 order by col0_3; --try and create mv with same name; create materialized view test_mv_3 as select * from test_source_3; -Failed to execute statement: PDB0009 - Materialized view already exists: test.test_mv_3 +Failed to execute statement: PDB1007 - Materialized view test.test_mv_3 already exists drop materialized view test_mv_3; 0 rows returned select * from test_mv3 order by col0_3; -Failed to execute statement: PDB0002 - Table 'test.test_mv3' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_mv3' doesn't exist drop materialized view test_mv_2; 0 rows returned select * from test_mv2 order by col0_2; -Failed to execute statement: PDB0002 - Table 'test.test_mv2' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_mv2' doesn't exist drop materialized view test_mv_1; 0 rows returned select * from test_mv1 order by col0_1; -Failed to execute statement: PDB0002 - Table 'test.test_mv1' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_mv1' doesn't exist --recreate mv; create materialized view test_mv_1 as select * from test_source_1; @@ -301,7 +301,7 @@ drop materialized view test_mv_1; --try and drop it again; drop materialized view test_mv_1; -Failed to execute statement: PDB0006 - Unknown materialized view: test.test_mv_1 +Failed to execute statement: PDB1003 - Unknown materialized view: test.test_mv_1 drop source test_source_2; 0 rows returned diff --git a/sqltest/testdata/multiple_source_test_out.txt b/sqltest/testdata/multiple_source_test_out.txt index 0a859501..0e4572b3 100644 --- a/sqltest/testdata/multiple_source_test_out.txt +++ b/sqltest/testdata/multiple_source_test_out.txt @@ -266,22 +266,22 @@ create source test_source_3( v6 ) ); -Failed to execute statement: PDB0008 - Source already exists: test.test_source_3 +Failed to execute statement: PDB1006 - Source test.test_source_3 already exists drop source test_source_3; 0 rows returned select * from test_source_3 order by col0_3; -Failed to execute statement: PDB0002 - Table 'test.test_source_3' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_source_3' doesn't exist drop source test_source_2; 0 rows returned select * from test_source_2 order by col0_2; -Failed to execute statement: PDB0002 - Table 'test.test_source_2' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_source_2' doesn't exist drop source test_source_1; 0 rows returned select * from test_source_1 order by col0_1; -Failed to execute statement: PDB0002 - Table 'test.test_source_1' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_source_1' doesn't exist --recreate source; create source test_source_1( @@ -346,7 +346,7 @@ drop source test_source_1; --try and drop it again; drop source test_source_1; -Failed to execute statement: PDB0005 - Unknown source: test.test_source_1 +Failed to execute statement: PDB1002 - Unknown source: test.test_source_1 --delete topic testtopic3; --delete topic testtopic2; diff --git a/sqltest/testdata/on_delete_test_out.txt b/sqltest/testdata/on_delete_test_out.txt index 6593c0dc..0bc3c031 100644 --- a/sqltest/testdata/on_delete_test_out.txt +++ b/sqltest/testdata/on_delete_test_out.txt @@ -62,7 +62,7 @@ select * from test_source_1 order by col0; --activate failpoint create_mv_1 -1; create materialized view test_mv_1 as select * from test_source_1; -Failed to execute statement: PDB0000 - Internal error - reference: 97ec17be-4eb8-44de-9908-d7888de7d56b please consult server logs for details +Failed to execute statement: PDB5000 - Internal error - reference: 649cc908-35f5-4764-aaff-b39daead1a05 please consult server logs for details --deactivate failpoint create_mv_1 -1; --restart cluster; @@ -72,7 +72,7 @@ use test; -- MV shouldn't exist; select * from test_mv_1; -Failed to execute statement: PDB0002 - Table 'test.test_mv_1' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_mv_1' doesn't exist -- trigger a failure after MV has been filled and persisted to storage but before to_delete rows have been removed; -- at the end of the create MV process. In this case the MV should exist after restart with all it's data as we don't delete from the; @@ -81,7 +81,7 @@ Failed to execute statement: PDB0002 - Table 'test.test_mv_1' doesn't exist --activate failpoint create_mv_2 -1; create materialized view test_mv_2 as select * from test_source_1; -Failed to execute statement: PDB0000 - Internal error - reference: 9c7cecd9-d64e-4204-b489-67a460d8fd8a please consult server logs for details +Failed to execute statement: PDB5000 - Internal error - reference: 1c5b49e8-5a77-4bf9-b5bf-ac7c36a536a7 please consult server logs for details --deactivate failpoint create_mv_2 -1; --restart cluster; @@ -116,7 +116,7 @@ drop materialized view test_mv_2; --activate failpoint fill_to_1 2; create materialized view test_mv_3 as select * from test_source_1; -Failed to execute statement: PDB0000 - Internal error - reference: 8989f8b0-af5c-4f88-95c9-63f6f2584cec please consult server logs for details +Failed to execute statement: PDB5000 - Internal error - reference: acf2df27-ccd1-43cd-889c-900b46e5d9cf please consult server logs for details --deactivate failpoint fill_to_1 2; --restart cluster; @@ -126,7 +126,7 @@ use test; -- MV shouldn't exist; select * from test_mv_3; -Failed to execute statement: PDB0002 - Table 'test.test_mv_3' doesn't exist +Failed to execute statement: PDB1000 - Table 'test.test_mv_3' doesn't exist drop source test_source_1; 0 rows returned diff --git a/sqltest/testdata/pull_limit_test_out.txt b/sqltest/testdata/pull_limit_test_out.txt index 58851b53..83627382 100644 --- a/sqltest/testdata/pull_limit_test_out.txt +++ b/sqltest/testdata/pull_limit_test_out.txt @@ -237,12 +237,12 @@ select col2 from test_source_1 limit 999999999; +----------------------+ | col2 | +----------------------+ -Failed to execute statement: PDB0002 - limit count cannot be larger than 50000 +Failed to execute statement: PDB1000 - Limit count cannot be larger than 50000 select col1 from test_source_1 order by col1 limit 999999999; +----------------------+ | col1 | +----------------------+ -Failed to execute statement: PDB0002 - limit count cannot be larger than 50000 +Failed to execute statement: PDB1000 - Limit count cannot be larger than 50000 -- explicit zero offset; select col2 from test_source_1 limit 0, 1; @@ -279,12 +279,12 @@ select col2 from test_source_1 limit 1 offset 10; +----------------------+ | col2 | +----------------------+ -Failed to execute statement: PDB0002 - offset must be zero +Failed to execute statement: PDB1000 - Offset must be zero select col1 from test_source_1 order by col1 limit 1 offset 10; +----------------------+ | col1 | +----------------------+ -Failed to execute statement: PDB0002 - offset must be zero +Failed to execute statement: PDB1000 - Offset must be zero --test with mv; @@ -556,12 +556,12 @@ select col2 from test_mv_1 limit 999999999; +----------------------+ | col2 | +----------------------+ -Failed to execute statement: PDB0002 - limit count cannot be larger than 50000 +Failed to execute statement: PDB1000 - Limit count cannot be larger than 50000 select col1 from test_mv_1 order by col1 limit 999999999; +----------------------+ | col1 | +----------------------+ -Failed to execute statement: PDB0002 - limit count cannot be larger than 50000 +Failed to execute statement: PDB1000 - Limit count cannot be larger than 50000 -- explicit zero offset; select col2 from test_mv_1 limit 0, 1; @@ -598,12 +598,12 @@ select col2 from test_mv_1 limit 1 offset 10; +----------------------+ | col2 | +----------------------+ -Failed to execute statement: PDB0002 - offset must be zero +Failed to execute statement: PDB1000 - Offset must be zero select col1 from test_mv_1 order by col1 limit 1 offset 10; +----------------------+ | col1 | +----------------------+ -Failed to execute statement: PDB0002 - offset must be zero +Failed to execute statement: PDB1000 - Offset must be zero drop materialized view test_mv_1; 0 rows returned diff --git a/sqltest/testdata/show_schemas_test_out.txt b/sqltest/testdata/show_schemas_test_out.txt index 06dc7f87..a0a648ed 100644 --- a/sqltest/testdata/show_schemas_test_out.txt +++ b/sqltest/testdata/show_schemas_test_out.txt @@ -34,13 +34,19 @@ show schemas; create source payments ( payment_id varchar, customer_id bigint, - amount decimal(10, 2) + amount decimal(10, 2), + primary key(payment_id) ) with ( brokername = "testbroker", topicname = "payments", headerencoding = "stringbytes", keyencoding = "stringbytes", - valueencoding = "json" + valueencoding = "json", + columnselectors = ( + v0, + v1, + v2 + ) ); 0 rows returned diff --git a/sqltest/testdata/show_schemas_test_script.txt b/sqltest/testdata/show_schemas_test_script.txt index 23b4d1b6..1cef5f60 100644 --- a/sqltest/testdata/show_schemas_test_script.txt +++ b/sqltest/testdata/show_schemas_test_script.txt @@ -12,13 +12,19 @@ show schemas; create source payments ( payment_id varchar, customer_id bigint, - amount decimal(10, 2) + amount decimal(10, 2), + primary key(payment_id) ) with ( brokername = "testbroker", topicname = "payments", headerencoding = "stringbytes", keyencoding = "stringbytes", - valueencoding = "json" + valueencoding = "json", + columnselectors = ( + v0, + v1, + v2 + ) ); --now go to another schema - test0 should still exist as it's not empty; diff --git a/sqltest/testdata/source_ingest_filter_test_out.txt b/sqltest/testdata/source_ingest_filter_test_out.txt index af493c4e..823ecaa0 100644 --- a/sqltest/testdata/source_ingest_filter_test_out.txt +++ b/sqltest/testdata/source_ingest_filter_test_out.txt @@ -77,5 +77,5 @@ create source test_source_1( v6 ) ); -Failed to execute statement: PDB0025 - invalid ingest filter "wtf is this?" +Failed to execute statement: PDB1000 - invalid ingest filter "wtf is this?" --delete topic testtopic; diff --git a/sqltest/testdata/use_test_out.txt b/sqltest/testdata/use_test_out.txt index 23981174..d1ce6ffb 100644 --- a/sqltest/testdata/use_test_out.txt +++ b/sqltest/testdata/use_test_out.txt @@ -271,11 +271,11 @@ select * from test_mv_3 order by col0; --should fail; select * from test_source_2 order by col0; -Failed to execute statement: PDB0002 - Table 'test2.test_source_2' doesn't exist +Failed to execute statement: PDB1000 - Table 'test2.test_source_2' doesn't exist select * from test_mv_2 order by col0; -Failed to execute statement: PDB0002 - Table 'test2.test_mv_2' doesn't exist +Failed to execute statement: PDB1000 - Table 'test2.test_mv_2' doesn't exist drop materialized view test_source_2; -Failed to execute statement: PDB0006 - Unknown materialized view: test2.test_source_2 +Failed to execute statement: PDB1003 - Unknown materialized view: test2.test_source_2 use test1; 0 rows returned @@ -315,11 +315,11 @@ select * from test_mv_1 order by col0; --should fail; select * from test_source_3 order by col0; -Failed to execute statement: PDB0002 - Table 'test1.test_source_3' doesn't exist +Failed to execute statement: PDB1000 - Table 'test1.test_source_3' doesn't exist select * from test_mv_3 order by col0; -Failed to execute statement: PDB0002 - Table 'test1.test_mv_3' doesn't exist +Failed to execute statement: PDB1000 - Table 'test1.test_mv_3' doesn't exist drop materialized view test_mv_3; -Failed to execute statement: PDB0006 - Unknown materialized view: test1.test_mv_3 +Failed to execute statement: PDB1003 - Unknown materialized view: test1.test_mv_3 use test2; 0 rows returned @@ -391,9 +391,9 @@ select * from test_mv_3 order by col0; --should fail; select * from test_source_2 order by col0; -Failed to execute statement: PDB0002 - Table 'test2.test_source_2' doesn't exist +Failed to execute statement: PDB1000 - Table 'test2.test_source_2' doesn't exist select * from test_mv_2 order by col0; -Failed to execute statement: PDB0002 - Table 'test2.test_mv_2' doesn't exist +Failed to execute statement: PDB1000 - Table 'test2.test_mv_2' doesn't exist drop materialized view test_mv_3; 0 rows returned