diff --git a/go.mod b/go.mod index 3b5cde065bab..4865741997c2 100644 --- a/go.mod +++ b/go.mod @@ -36,12 +36,12 @@ require ( github.com/cockroachdb/cockroach-go v0.0.0-20200504194139-73ffeee90b62 github.com/cockroachdb/crlfmt v0.0.0-20200116191136-a78e1c207bc0 github.com/cockroachdb/datadriven v1.0.1-0.20200826112548-92602d883b11 - github.com/cockroachdb/errors v1.7.3 + github.com/cockroachdb/errors v1.7.4 github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55 github.com/cockroachdb/gostdlib v1.13.0 github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f github.com/cockroachdb/pebble v0.0.0-20200831143935-e6a9f9a3c936 - github.com/cockroachdb/redact v1.0.2 + github.com/cockroachdb/redact v1.0.5 github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 github.com/cockroachdb/stress v0.0.0-20170808184505-29b5d31b4c3a diff --git a/go.sum b/go.sum index 63cce6d9483b..89a2ece5cc1d 100644 --- a/go.sum +++ b/go.sum @@ -148,8 +148,8 @@ github.com/cockroachdb/datadriven v1.0.1-0.20200826112548-92602d883b11/go.mod h1 github.com/cockroachdb/errors v1.2.4 h1:Lap807SXTH5tri2TivECb/4abUkMZC9zRoLarvcKDqs= github.com/cockroachdb/errors v1.2.4/go.mod h1:rQD95gz6FARkaKkQXUksEje/d9a6wBJoCr5oaCLELYA= github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= -github.com/cockroachdb/errors v1.7.3 h1:3Vx4uIENFw44XbZe4Gnp/Soh4PlSowP7DaDu7INSKzE= -github.com/cockroachdb/errors v1.7.3/go.mod h1:pKPbzdh8tCRjLCJUFHEoRjfhT8HvwyW2cxImC+zVlMM= +github.com/cockroachdb/errors v1.7.4 h1:ZlMG24ZIxz6Yp00gR3KkJKsMEWWNyeOWiGyaUav5y8o= +github.com/cockroachdb/errors v1.7.4/go.mod h1:ngNOpuFdFU8EOjSsuVmCU9AODd7yp3XXEr0JUjbCN5U= github.com/cockroachdb/etcd v0.4.7-0.20200615211340-a17df30d5955 h1:1ELogBFqZl2Cj/Rn0wWAU4z2/ghBlHzhXDlbT8RlrKU= github.com/cockroachdb/etcd v0.4.7-0.20200615211340-a17df30d5955/go.mod h1:Vshs83p1UXI6fjU8eWVgqXeewFxRDP1fNVF5PHiSBm0= github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55 h1:YqzBA7tf8Gv8Oz0BbBsPenqkyjiohS7EUIwi7p1QJCU= @@ -166,8 +166,8 @@ github.com/cockroachdb/pebble v0.0.0-20200831143935-e6a9f9a3c936 h1:MaPvrDBbbWMz github.com/cockroachdb/pebble v0.0.0-20200831143935-e6a9f9a3c936/go.mod h1:hU7vhtrqonEphNF+xt8/lHdaBprxmV1h8BOGrd9XwmQ= github.com/cockroachdb/redact v0.0.0-20200622112456-cd282804bbd3 h1:2+dpIJzYMSbLi0587YXpi8tOJT52qCOI/1I0UNThc/I= github.com/cockroachdb/redact v0.0.0-20200622112456-cd282804bbd3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= -github.com/cockroachdb/redact v1.0.2 h1:bRktqHBXPqI+9bkOx0ikn9RS09G9k83oAdLx6rXRVTQ= -github.com/cockroachdb/redact v1.0.2/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/redact v1.0.5 h1:yxqIMS6G2Bvi6GiSHFmsrFGO3aP1rwt8cOm4pixw9eY= +github.com/cockroachdb/redact v1.0.5/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd h1:KFOt5I9nEKZgCnOSmy8r4Oykh8BYQO8bFOTgHDS8YZA= github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd/go.mod h1:AN708GD2FFeLgUHMbD58YPe4Nw8GG//3rwgyG4L9gR0= github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= diff --git a/pkg/ccl/changefeedccl/avro.go b/pkg/ccl/changefeedccl/avro.go index 79839a5f7e77..c9f57c52f507 100644 --- a/pkg/ccl/changefeedccl/avro.go +++ b/pkg/ccl/changefeedccl/avro.go @@ -10,7 +10,6 @@ package changefeedccl import ( "encoding/json" - "fmt" "math/big" "time" @@ -86,7 +85,7 @@ func avroUnionKey(t avroSchemaType) string { case *avroRecord: return s.Name default: - panic(fmt.Sprintf(`unsupported type %T %v`, t, t)) + panic(errors.AssertionFailedf(`unsupported type %T %v`, t, t)) } } diff --git a/pkg/col/coldataext/datum_vec.go b/pkg/col/coldataext/datum_vec.go index e1e6854f73b6..5eeeb41280e3 100644 --- a/pkg/col/coldataext/datum_vec.go +++ b/pkg/col/coldataext/datum_vec.go @@ -11,8 +11,6 @@ package coldataext import ( - "fmt" - "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" @@ -20,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/errors" ) // Datum wraps a tree.Datum. This is the struct that datumVec.Get() returns. @@ -175,7 +174,7 @@ func (dv *datumVec) assertValidDatum(datum tree.Datum) { func (dv *datumVec) assertSameTypeFamily(t *types.T) { if dv.t.Family() != t.Family() { colexecerror.InternalError( - fmt.Sprintf("cannot use value of type %+v on a datumVec of type %+v", t, dv.t), + errors.AssertionFailedf("cannot use value of type %+v on a datumVec of type %+v", t, dv.t), ) } } @@ -201,7 +200,7 @@ func maybeUnwrapDatum(v coldata.Datum) tree.Datum { } else if datum, ok := v.(tree.Datum); ok { return datum } - colexecerror.InternalError(fmt.Sprintf("unexpected value: %v", v)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected value: %v", v)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/col/coldatatestutils/random_testutils.go b/pkg/col/coldatatestutils/random_testutils.go index e82113ee2e1b..14bcf241ef3f 100644 --- a/pkg/col/coldatatestutils/random_testutils.go +++ b/pkg/col/coldatatestutils/random_testutils.go @@ -12,7 +12,6 @@ package coldatatestutils import ( "context" - "fmt" "math/rand" "time" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" ) // maxVarLen specifies a length limit for variable length types (e.g. byte slices). @@ -228,7 +228,7 @@ func RandomBatch( // less than batchSize. func RandomSel(rng *rand.Rand, batchSize int, probOfOmitting float64) []int { if probOfOmitting < 0 || probOfOmitting > 1 { - colexecerror.InternalError(fmt.Sprintf("probability of omitting a row is %f - outside of [0, 1] range", probOfOmitting)) + colexecerror.InternalError(errors.AssertionFailedf("probability of omitting a row is %f - outside of [0, 1] range", probOfOmitting)) } sel := make([]int, 0, batchSize) for i := 0; i < batchSize; i++ { @@ -396,7 +396,7 @@ func (o *RandomDataOp) ChildCount(verbose bool) int { // Child implements the execinfra.OpNode interface. func (o *RandomDataOp) Child(nth int, verbose bool) execinfra.OpNode { - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index ea38b1bdf528..dfdde110f418 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -1186,7 +1186,7 @@ func TestJobLifecycle(t *testing.T) { t.Run("bad job details fail", func(t *testing.T) { defer func() { - if r, ok := recover().(string); !ok || !strings.Contains(r, "unknown details type int") { + if r, ok := recover().(error); !ok || !strings.Contains(r.Error(), "unknown details type int") { t.Fatalf("expected 'unknown details type int', but got: %v", r) } }() diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index 63b78e8772dc..70622df98de9 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -11,11 +11,11 @@ package jobspb import ( - "fmt" "strings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/stats" + "github.com/cockroachdb/errors" ) // Details is a marker interface for job details proto structs. @@ -67,7 +67,7 @@ func DetailsType(d isPayload_Details) Type { case *Payload_TypeSchemaChange: return TypeTypeSchemaChange default: - panic(fmt.Sprintf("Payload.Type called on a payload with an unknown details type: %T", d)) + panic(errors.AssertionFailedf("Payload.Type called on a payload with an unknown details type: %T", d)) } } @@ -97,7 +97,7 @@ func WrapProgressDetails(details ProgressDetails) interface { case TypeSchemaChangeProgress: return &Progress_TypeSchemaChange{TypeSchemaChange: &d} default: - panic(fmt.Sprintf("WrapProgressDetails: unknown details type %T", d)) + panic(errors.AssertionFailedf("WrapProgressDetails: unknown details type %T", d)) } } @@ -185,7 +185,7 @@ func WrapPayloadDetails(details Details) interface { case TypeSchemaChangeDetails: return &Payload_TypeSchemaChange{TypeSchemaChange: &d} default: - panic(fmt.Sprintf("jobs.WrapPayloadDetails: unknown details type %T", d)) + panic(errors.AssertionFailedf("jobs.WrapPayloadDetails: unknown details type %T", d)) } } diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go index 26d60aa8e74d..af89c716a078 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go @@ -175,7 +175,7 @@ func (ds *DistSender) partialRangeFeed( case errors.HasType(err, (*roachpb.RangeFeedRetryError)(nil)): var t *roachpb.RangeFeedRetryError if ok := errors.As(err, &t); !ok { - panic(fmt.Sprintf("wrong error type: %T", err)) + panic(errors.AssertionFailedf("wrong error type: %T", err)) } switch t.Reason { case roachpb.RangeFeedRetryError_REASON_REPLICA_REMOVED, diff --git a/pkg/kv/kvserver/rangefeed/processor.go b/pkg/kv/kvserver/rangefeed/processor.go index 58c5632d089a..952582fbf882 100644 --- a/pkg/kv/kvserver/rangefeed/processor.go +++ b/pkg/kv/kvserver/rangefeed/processor.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/errors" ) const ( @@ -561,7 +562,7 @@ func (p *Processor) consumeLogicalOps(ctx context.Context, ops []enginepb.MVCCLo // No updates to publish. default: - panic(fmt.Sprintf("unknown logical op %T", t)) + panic(errors.AssertionFailedf("unknown logical op %T", t)) } // Determine whether the operation caused the resolved timestamp to diff --git a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go index b5c4ecf5ff11..98537bb4f712 100644 --- a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go +++ b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" ) // A rangefeed's "resolved timestamp" is defined as the timestamp at which no @@ -205,7 +206,7 @@ func (rts *resolvedTimestamp) consumeLogicalOp(op enginepb.MVCCLogicalOp) bool { return rts.intentQ.Del(t.TxnID) default: - panic(fmt.Sprintf("unknown logical op %T", t)) + panic(errors.AssertionFailedf("unknown logical op %T", t)) } } diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index 3c6ccfc84485..9f0dda92ed3f 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -464,7 +464,7 @@ func (r *Replica) populatePrevValsInLogicalOpLogRaftMuLocked( // Nothing to do. continue default: - panic(fmt.Sprintf("unknown logical op %T", t)) + panic(errors.AssertionFailedf("unknown logical op %T", t)) } // Don't read previous values from the reader for operations that are @@ -538,7 +538,7 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( // Nothing to do. continue default: - panic(fmt.Sprintf("unknown logical op %T", t)) + panic(errors.AssertionFailedf("unknown logical op %T", t)) } // Don't read values from the reader for operations that are not needed diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index dd627dd26aca..f65103126ccd 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -309,11 +309,7 @@ func CombineResponses(left, right Response) error { if lOK && rOK { return cLeft.combine(cRight) } else if lOK != rOK { - // TODO(knz): Simplify to %T when - // https://github.com/cockroachdb/cockroach/issues/53207 is - // addressed. - return errors.Errorf("can not combine %s and %s", - errors.Safe(fmt.Sprintf("%T", left)), errors.Safe(fmt.Sprintf("%T", right))) + return errors.Errorf("can not combine %T and %T", left, right) } return nil } @@ -606,7 +602,7 @@ func (sr *ReverseScanResponse) Verify(req Request) error { func (ru *RequestUnion) MustSetInner(args Request) { ru.Reset() if !ru.SetInner(args) { - panic(fmt.Sprintf("%T excludes %T", ru, args)) + panic(errors.AssertionFailedf("%T excludes %T", ru, args)) } } @@ -616,7 +612,7 @@ func (ru *RequestUnion) MustSetInner(args Request) { func (ru *ResponseUnion) MustSetInner(reply Response) { ru.Reset() if !ru.SetInner(reply) { - panic(fmt.Sprintf("%T excludes %T", ru, reply)) + panic(errors.AssertionFailedf("%T excludes %T", ru, reply)) } } @@ -1360,7 +1356,7 @@ func (b *BulkOpSummary) Add(other BulkOpSummary) { func (e *RangeFeedEvent) MustSetValue(value interface{}) { e.Reset() if !e.SetValue(value) { - panic(fmt.Sprintf("%T excludes %T", e, value)) + panic(errors.AssertionFailedf("%T excludes %T", e, value)) } } diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 571b674b5290..79e464dc3857 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1956,7 +1956,7 @@ func (l *Lease) Equal(that interface{}) bool { if ok { that1 = &that2 } else { - panic(fmt.Sprintf("attempting to compare lease to %T", that)) + panic(errors.AssertionFailedf("attempting to compare lease to %T", that)) } } if that1 == nil { diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index 5853713b4ca1..08427e6dd92e 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -48,8 +48,8 @@ type transactionRestartError interface { // ErrorUnexpectedlySet creates a string to panic with when a response (typically // a roachpb.BatchResponse) unexpectedly has Error set in its response header. -func ErrorUnexpectedlySet(culprit, response interface{}) string { - return fmt.Sprintf("error is unexpectedly set, culprit is %T:\n%+v", culprit, response) +func ErrorUnexpectedlySet(culprit, response interface{}) error { + return errors.AssertionFailedf("error is unexpectedly set, culprit is %T:\n%+v", culprit, response) } // ErrorPriority is used to rank errors such that the "best" one is chosen to be @@ -271,7 +271,7 @@ func (e *Error) SetDetail(err error) { if !e.Detail.SetInner(err) { _, isInternalError := err.(*internalError) if !isInternalError && e.TransactionRestart != TransactionRestart_NONE { - panic(fmt.Sprintf("transactionRestartError %T must be an ErrorDetail", err)) + panic(errors.AssertionFailedf("transactionRestartError %T must be an ErrorDetail", err)) } } e.checkTxnStatusValid() diff --git a/pkg/sql/colcontainer/partitionedqueue.go b/pkg/sql/colcontainer/partitionedqueue.go index ed96d46f2e7e..38a5f0b298a6 100644 --- a/pkg/sql/colcontainer/partitionedqueue.go +++ b/pkg/sql/colcontainer/partitionedqueue.go @@ -12,7 +12,6 @@ package colcontainer import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" @@ -144,7 +143,7 @@ func NewPartitionedDiskQueue( // DiskQueues cannot serialize zero length schemas, so catch this error // early. // TODO(asubiotto): We could support this, but not sure we need to. - colexecerror.InternalError("zero length schema unsupported") + colexecerror.InternalError(errors.AssertionFailedf("zero length schema unsupported")) } return &PartitionedDiskQueue{ typs: typs, @@ -175,7 +174,7 @@ func (p *PartitionedDiskQueue) closeWritePartition( ctx context.Context, idx int, releaseFDOption closeWritePartitionArgument, ) error { if p.partitions[idx].state != partitionStateWriting { - colexecerror.InternalError(fmt.Sprintf("illegal state change from %d to partitionStateClosedForWriting, only partitionStateWriting allowed", p.partitions[idx].state)) + colexecerror.InternalError(errors.AssertionFailedf("illegal state change from %d to partitionStateClosedForWriting, only partitionStateWriting allowed", p.partitions[idx].state)) } if err := p.partitions[idx].Enqueue(ctx, coldata.ZeroBatch); err != nil { return err @@ -190,7 +189,7 @@ func (p *PartitionedDiskQueue) closeWritePartition( func (p *PartitionedDiskQueue) closeReadPartition(idx int) error { if p.partitions[idx].state != partitionStateReading { - colexecerror.InternalError(fmt.Sprintf("illegal state change from %d to partitionStateClosedForReading, only partitionStateReading allowed", p.partitions[idx].state)) + colexecerror.InternalError(errors.AssertionFailedf("illegal state change from %d to partitionStateClosedForReading, only partitionStateReading allowed", p.partitions[idx].state)) } if err := p.partitions[idx].CloseRead(); err != nil { return err @@ -300,7 +299,7 @@ func (p *PartitionedDiskQueue) Dequeue( case partitionStatePermanentlyClosed: return errors.Errorf("partition at index %d permanently closed, cannot Dequeue", partitionIdx) default: - colexecerror.InternalError(fmt.Sprintf("unhandled state %d", state)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled state %d", state)) } notEmpty, err := p.partitions[idx].Dequeue(ctx, batch) if err != nil { @@ -317,7 +316,8 @@ func (p *PartitionedDiskQueue) Dequeue( // Dequeue but more batches will be added in the future (i.e. a zero batch // was never enqueued). Since we require partitions to be closed for writing // before reading, this state is unexpected. - colexecerror.InternalError("DiskQueue unexpectedly returned that more data will be added") + colexecerror.InternalError( + errors.AssertionFailedf("DiskQueue unexpectedly returned that more data will be added")) } return nil } diff --git a/pkg/sql/colexec/aggregate_funcs.go b/pkg/sql/colexec/aggregate_funcs.go index d470b31ad3e8..6a80a5cec164 100644 --- a/pkg/sql/colexec/aggregate_funcs.go +++ b/pkg/sql/colexec/aggregate_funcs.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // isAggOptimized returns whether aggFn has an optimized implementation. @@ -138,17 +139,17 @@ func (h *hashAggregateFuncBase) Reset() { } func (h *hashAggregateFuncBase) CurrentOutputIndex() int { - colexecerror.InternalError("CurrentOutputIndex called with hash aggregation") + colexecerror.InternalError(errors.AssertionFailedf("CurrentOutputIndex called with hash aggregation")) // This code is unreachable, but the compiler cannot infer that. return 0 } func (h *hashAggregateFuncBase) SetOutputIndex(int) { - colexecerror.InternalError("SetOutputIndex called with hash aggregation") + colexecerror.InternalError(errors.AssertionFailedf("SetOutputIndex called with hash aggregation")) } func (h *hashAggregateFuncBase) HandleEmptyInputScalar() { - colexecerror.InternalError("HandleEmptyInputScalar called with hash aggregation") + colexecerror.InternalError(errors.AssertionFailedf("HandleEmptyInputScalar called with hash aggregation")) } // aggregateFuncAlloc is an aggregate function allocator that pools allocations diff --git a/pkg/sql/colexec/and_or_projection.eg.go b/pkg/sql/colexec/and_or_projection.eg.go index 7328d177a732..98dbbd254de3 100644 --- a/pkg/sql/colexec/and_or_projection.eg.go +++ b/pkg/sql/colexec/and_or_projection.eg.go @@ -11,7 +11,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -161,7 +160,7 @@ func (o *andProjOp) Child(nth int, verbose bool) execinfra.OpNode { case 2: return o.rightProjOpChain default: - colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -599,7 +598,7 @@ func (o *andRightNullProjOp) Child(nth int, verbose bool) execinfra.OpNode { case 2: return o.rightProjOpChain default: - colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -1004,7 +1003,7 @@ func (o *andLeftNullProjOp) Child(nth int, verbose bool) execinfra.OpNode { case 2: return o.rightProjOpChain default: - colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -1390,7 +1389,7 @@ func (o *orProjOp) Child(nth int, verbose bool) execinfra.OpNode { case 2: return o.rightProjOpChain default: - colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -1829,7 +1828,7 @@ func (o *orRightNullProjOp) Child(nth int, verbose bool) execinfra.OpNode { case 2: return o.rightProjOpChain default: - colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -2235,7 +2234,7 @@ func (o *orLeftNullProjOp) Child(nth int, verbose bool) execinfra.OpNode { case 2: return o.rightProjOpChain default: - colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/and_or_projection_tmpl.go b/pkg/sql/colexec/and_or_projection_tmpl.go index 1ebf3e099e31..3d46c977fe89 100644 --- a/pkg/sql/colexec/and_or_projection_tmpl.go +++ b/pkg/sql/colexec/and_or_projection_tmpl.go @@ -21,7 +21,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -136,7 +135,7 @@ func (o *_OP_LOWERProjOp) Child(nth int, verbose bool) execinfra.OpNode { case 2: return o.rightProjOpChain default: - colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/avg_agg_tmpl.go b/pkg/sql/colexec/avg_agg_tmpl.go index 75056a9e651b..0a11bbb0793a 100644 --- a/pkg/sql/colexec/avg_agg_tmpl.go +++ b/pkg/sql/colexec/avg_agg_tmpl.go @@ -36,13 +36,13 @@ import ( // input to the result of the second input / the third input, where the third // input is an int64. func _ASSIGN_DIV_INT64(_, _, _, _, _, _ string) { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // _ASSIGN_ADD is the template addition function for assigning the first input // to the result of the second input + the third input. func _ASSIGN_ADD(_, _, _, _, _, _ string) { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/bool_and_or_agg_tmpl.go b/pkg/sql/colexec/bool_and_or_agg_tmpl.go index b946b876e5f8..84dea2be5336 100644 --- a/pkg/sql/colexec/bool_and_or_agg_tmpl.go +++ b/pkg/sql/colexec/bool_and_or_agg_tmpl.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/errors" ) // Remove unused warning. @@ -36,7 +37,7 @@ var _ = colexecerror.InternalError // first input to the result of a boolean operation of the second and the third // inputs. func _ASSIGN_BOOL_OP(_, _, _ string) { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/case.go b/pkg/sql/colexec/case.go index 3931d5c7ea63..6c7589c7b83b 100644 --- a/pkg/sql/colexec/case.go +++ b/pkg/sql/colexec/case.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -20,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) type caseOp struct { @@ -60,7 +60,7 @@ func (c *caseOp) Child(nth int, verbose bool) execinfra.OpNode { } else if nth == 1+len(c.caseOps) { return c.elseOp } - colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/cast_tmpl.go b/pkg/sql/colexec/cast_tmpl.go index 1111c6460765..9ce0eb942d8e 100644 --- a/pkg/sql/colexec/cast_tmpl.go +++ b/pkg/sql/colexec/cast_tmpl.go @@ -56,17 +56,17 @@ const _RIGHT_CANONICAL_TYPE_FAMILY = types.UnknownFamily const _RIGHT_TYPE_WIDTH = 0 func _CAST(to, from, fromCol interface{}) { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // This will be replaced with execgen.SET. func _R_SET(to, from interface{}) { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // This will be replaced with execgen.SLICE. func _L_SLICE(col, i, j interface{}) interface{} { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 805fcf940c8c..d21aa451f04e 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -531,7 +531,7 @@ func NewColOperator( result.OpMonitors = result.OpMonitors[:0] } if panicErr != nil { - colexecerror.InternalError(panicErr) + colexecerror.InternalError(log.PanicAsError(0, panicErr)) } }() spec := args.Spec @@ -2008,7 +2008,7 @@ func planLogicalProjectionOp( typedLeft = t.TypedLeft() typedRight = t.TypedRight() default: - colexecerror.InternalError(fmt.Sprintf("unexpected logical expression type %s", t.String())) + colexecerror.InternalError(errors.AssertionFailedf("unexpected logical expression type %s", t.String())) } leftProjOpChain, leftIdx, typs, internalMemUsedLeft, err = planProjectionOperators( ctx, evalCtx, typedLeft, typs, leftFeedOp, acc, factory, diff --git a/pkg/sql/colexec/colbuilder/window_functions_util.go b/pkg/sql/colexec/colbuilder/window_functions_util.go index e7e9c8f7d1da..4c66625bd561 100644 --- a/pkg/sql/colexec/colbuilder/window_functions_util.go +++ b/pkg/sql/colexec/colbuilder/window_functions_util.go @@ -11,10 +11,9 @@ package colbuilder import ( - "fmt" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/errors" ) // SupportedWindowFns contains all window functions supported by the @@ -45,7 +44,7 @@ func windowFnNeedsPeersInfo(windowFn execinfrapb.WindowerSpec_WindowFunc) bool { execinfrapb.WindowerSpec_CUME_DIST: return true default: - colexecerror.InternalError(fmt.Sprintf("window function %s is not supported", windowFn.String())) + colexecerror.InternalError(errors.AssertionFailedf("window function %s is not supported", windowFn.String())) // This code is unreachable, but the compiler cannot infer that. return false } diff --git a/pkg/sql/colexec/columnarizer.go b/pkg/sql/colexec/columnarizer.go index 8aa2cda54cb8..e6fbd1a38acd 100644 --- a/pkg/sql/colexec/columnarizer.go +++ b/pkg/sql/colexec/columnarizer.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -22,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // Columnarizer turns an execinfra.RowSource input into an Operator output, by @@ -147,7 +147,7 @@ func (c *Columnarizer) Next(context.Context) coldata.Batch { // Columnarizers are not expected to be Run, so we prohibit calling this method // on them. func (c *Columnarizer) Run(context.Context) { - colexecerror.InternalError("Columnarizer should not be Run") + colexecerror.InternalError(errors.AssertionFailedf("Columnarizer should not be Run")) } var ( @@ -189,9 +189,9 @@ func (c *Columnarizer) Child(nth int, verbose bool) execinfra.OpNode { if n, ok := c.input.(execinfra.OpNode); ok { return n } - colexecerror.InternalError("input to Columnarizer is not an execinfra.OpNode") + colexecerror.InternalError(errors.AssertionFailedf("input to Columnarizer is not an execinfra.OpNode")) } - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/disk_spiller.go b/pkg/sql/colexec/disk_spiller.go index beb391a29b69..fd7efc20eb70 100644 --- a/pkg/sql/colexec/disk_spiller.go +++ b/pkg/sql/colexec/disk_spiller.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "strings" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -20,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" + "github.com/cockroachdb/errors" ) // oneInputDiskSpiller is an Operator that manages the fallback from a one @@ -278,7 +278,7 @@ func (d *diskSpillerBase) Child(nth int, verbose bool) execinfra.OpNode { case 0: return d.inMemoryOp default: - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/distinct.eg.go b/pkg/sql/colexec/distinct.eg.go index 1c91e41cba36..db32411e2d48 100644 --- a/pkg/sql/colexec/distinct.eg.go +++ b/pkg/sql/colexec/distinct.eg.go @@ -51,7 +51,7 @@ func OrderedDistinctColsToOperators( } } if r, ok = input.(ResettableOperator); !ok { - colexecerror.InternalError("unexpectedly an ordered distinct is not a resetter") + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly an ordered distinct is not a resetter")) } distinctChain := &distinctChainOps{ ResettableOperator: r, diff --git a/pkg/sql/colexec/distinct_tmpl.go b/pkg/sql/colexec/distinct_tmpl.go index 88311a2d51ae..1e3dd23acd23 100644 --- a/pkg/sql/colexec/distinct_tmpl.go +++ b/pkg/sql/colexec/distinct_tmpl.go @@ -55,7 +55,7 @@ func OrderedDistinctColsToOperators( } } if r, ok = input.(ResettableOperator); !ok { - colexecerror.InternalError("unexpectedly an ordered distinct is not a resetter") + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly an ordered distinct is not a resetter")) } distinctChain := &distinctChainOps{ ResettableOperator: r, @@ -97,7 +97,7 @@ type _GOTYPESLICE interface{} // _ASSIGN_NE is the template equality function for assigning the first input // to the result of the second input != the third input. func _ASSIGN_NE(_ bool, _, _, _, _, _ _GOTYPE) bool { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // _CANONICAL_TYPE_FAMILY is the template variable. diff --git a/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go index eb92a81458de..c6ad7b8cd5e3 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) type avgTmplInfo struct { @@ -58,7 +59,7 @@ func (a avgTmplInfo) AssignDivInt64(targetElem, leftElem, rightElem, _, _, _ str case toVecMethod(types.IntervalFamily, anyWidth): return fmt.Sprintf("%s = %s.Div(int64(%s))", targetElem, leftElem, rightElem) } - colexecerror.InternalError("unsupported avg agg type") + colexecerror.InternalError(errors.AssertionFailedf("unsupported avg agg type")) // This code is unreachable, but the compiler cannot infer that. return "" } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/bool_and_or_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/bool_and_or_agg_gen.go index aac7ce753c96..dba81aa3fa91 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/bool_and_or_agg_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/bool_and_or_agg_gen.go @@ -17,6 +17,7 @@ import ( "text/template" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/errors" ) type booleanAggTmplInfo struct { @@ -30,7 +31,7 @@ func (b booleanAggTmplInfo) AssignBoolOp(target, l, r string) string { case false: return fmt.Sprintf("%s = %s || %s", target, l, r) default: - colexecerror.InternalError("unsupported boolean agg type") + colexecerror.InternalError(errors.AssertionFailedf("unsupported boolean agg type")) // This code is unreachable, but the compiler cannot infer that. return "" } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/hashtable_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/hashtable_gen.go index 132107d11466..d8f11754f44e 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/hashtable_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/hashtable_gen.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/errors" ) // hashTableMode describes the mode in which the hash table can operate. @@ -52,7 +53,7 @@ func (m hashTableMode) String() string { case hashTableFullBuildDeletingProbe: return "full_deleting" default: - colexecerror.InternalError("unexpected hashTableMode") + colexecerror.InternalError(errors.AssertionFailedf("unexpected hashTableMode")) // This code is unreachable, but the compiler cannot infer that. return "" } @@ -138,7 +139,7 @@ func genHashTable(inputFileContents string, wr io.Writer, htm hashTableMode) err } } if data == nil { - colexecerror.InternalError("unexpectedly didn't find overload for tree.NE") + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly didn't find overload for tree.NE")) } return tmpl.Execute(wr, struct { Overloads interface{} diff --git a/pkg/sql/colexec/execgen/cmd/execgen/main.go b/pkg/sql/colexec/execgen/cmd/execgen/main.go index 1a926718c9d6..88b92b886d00 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/main.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/main.go @@ -63,7 +63,7 @@ type entry struct { func registerGenerator(g generator, outputFile, inputFile string) { if _, ok := generators[outputFile]; ok { - colexecerror.InternalError(fmt.Sprintf("%s generator already registered", outputFile)) + colexecerror.InternalError(errors.AssertionFailedf("%s generator already registered", outputFile)) } generators[outputFile] = entry{fn: g, inputFile: inputFile} } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go index dfc20339eb6f..ca7d60800e2d 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // overloadBase and other overload-related structs form a leveled hierarchy @@ -429,7 +430,7 @@ func goTypeSliceName(canonicalTypeFamily types.Family, width int32) string { case 64, anyWidth: return "coldata.Int64s" default: - colexecerror.InternalError(fmt.Sprintf("unexpected int width %d", width)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected int width %d", width)) // This code is unreachable, but the compiler cannot infer that. return "" } @@ -442,7 +443,7 @@ func goTypeSliceName(canonicalTypeFamily types.Family, width int32) string { case typeconv.DatumVecCanonicalTypeFamily: return "coldata.DatumVec" } - colexecerror.InternalError(fmt.Sprintf("unsupported canonical type family %s", canonicalTypeFamily)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported canonical type family %s", canonicalTypeFamily)) return "" } @@ -845,7 +846,7 @@ func toVecMethod(canonicalTypeFamily types.Family, width int32) string { case 64, anyWidth: return "Int64" default: - colexecerror.InternalError(fmt.Sprintf("unexpected width of int type family: %d", width)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected width of int type family: %d", width)) } case types.FloatFamily: return "Float64" @@ -856,7 +857,7 @@ func toVecMethod(canonicalTypeFamily types.Family, width int32) string { case typeconv.DatumVecCanonicalTypeFamily: return "Datum" default: - colexecerror.InternalError(fmt.Sprintf("unsupported canonical type family %s", canonicalTypeFamily)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported canonical type family %s", canonicalTypeFamily)) } // This code is unreachable, but the compiler cannot infer that. return "" @@ -882,7 +883,7 @@ func toPhysicalRepresentation(canonicalTypeFamily types.Family, width int32) str case 64, anyWidth: return "int64" default: - colexecerror.InternalError(fmt.Sprintf("unexpected width of int type family: %d", width)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected width of int type family: %d", width)) } case types.FloatFamily: return "float64" @@ -896,7 +897,7 @@ func toPhysicalRepresentation(canonicalTypeFamily types.Family, width int32) str // different packages (sql/colexec and col/coldata). return "interface{}" default: - colexecerror.InternalError(fmt.Sprintf("unsupported canonical type family %s", canonicalTypeFamily)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported canonical type family %s", canonicalTypeFamily)) } // This code is unreachable, but the compiler cannot infer that. return "" diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go index 947eff50366b..cdb1ec30ceec 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) var binaryOpDecMethod = map[tree.BinaryOperator]string{ @@ -278,7 +279,7 @@ func (bytesCustomizer) getBinOpAssignFunc() assignFunc { } `, leftElem, rightElem, set(types.BytesFamily, caller, idx, "r")) } else { - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } return result } @@ -415,7 +416,7 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { upperBound = "math.MaxInt32" lowerBound = "math.MinInt32" default: - colexecerror.InternalError(fmt.Sprintf("unhandled integer width %d", c.width)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled integer width %d", c.width)) } args["UpperBound"] = upperBound @@ -495,7 +496,7 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { `, execgen.BinaryOpName[binOp]))) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.OpStr)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.OpStr)) } if err := t.Execute(&buf, args); err != nil { @@ -582,7 +583,7 @@ func (c timestampCustomizer) getBinOpAssignFunc() assignFunc { `, targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } // This code is unreachable, but the compiler cannot infer that. return "" @@ -599,7 +600,7 @@ func (c intervalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = %[2]s.Sub(%[3]s)`, targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } // This code is unreachable, but the compiler cannot infer that. return "" @@ -616,7 +617,7 @@ func (c timestampIntervalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = duration.Add(%[2]s, %[3]s.Mul(-1))`, targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } // This code is unreachable, but the compiler cannot infer that. return "" @@ -630,7 +631,7 @@ func (c intervalTimestampCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = duration.Add(%[3]s, %[2]s)`, targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } return "" } @@ -650,7 +651,7 @@ func (c intervalIntCustomizer) getBinOpAssignFunc() assignFunc { %[1]s = %[2]s.Div(int64(%[3]s))`, targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } return "" } @@ -663,7 +664,7 @@ func (c intIntervalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = %[3]s.Mul(int64(%[2]s))`, targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } return "" } @@ -683,7 +684,7 @@ func (c intervalFloatCustomizer) getBinOpAssignFunc() assignFunc { %[1]s = %[2]s.DivFloat(float64(%[3]s))`, targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } return "" } @@ -696,7 +697,7 @@ func (c floatIntervalCustomizer) getBinOpAssignFunc() assignFunc { return fmt.Sprintf(`%[1]s = %[3]s.MulFloat(float64(%[2]s))`, targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } return "" } @@ -714,7 +715,7 @@ func (c intervalDecimalCustomizer) getBinOpAssignFunc() assignFunc { %[1]s = %[2]s.MulFloat(f)`, targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } return "" } @@ -733,7 +734,7 @@ func (c decimalIntervalCustomizer) getBinOpAssignFunc() assignFunc { targetElem, leftElem, rightElem) default: - colexecerror.InternalError(fmt.Sprintf("unhandled binary operator %s", op.overloadBase.BinOp.String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled binary operator %s", op.overloadBase.BinOp.String())) } return "" } @@ -804,7 +805,7 @@ func convertNativeToDatum( runtimeConversion = fmt.Sprintf("tree.DBytes(%s)", nativeElem) } default: - colexecerror.InternalError(fmt.Sprintf("unexpected canonical type family: %s", canonicalTypeFamily)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected canonical type family: %s", canonicalTypeFamily)) } return fmt.Sprintf(` _convertedNativeElem := %[1]s @@ -855,7 +856,7 @@ func parseNonIndexableTargetElem(targetElem string) (caller string, index string // the closing square bracket. tokens := strings.Split(targetElem[:len(targetElem)-1], "[") if len(tokens) != 2 { - colexecerror.InternalError("unexpectedly len(tokens) != 2") + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly len(tokens) != 2")) } caller = tokens[0] index = tokens[1] diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_cast.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_cast.go index ac2cbb4378ce..624518a87a00 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_cast.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_cast.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) var castableCanonicalTypeFamilies = make(map[types.Family][]types.Family) @@ -153,7 +154,7 @@ var castTypeCustomizers = make(map[typePair]typeCustomizer) func registerCastTypeCustomizer(pair typePair, customizer typeCustomizer) { if _, found := castTypeCustomizers[pair]; found { - colexecerror.InternalError(fmt.Sprintf("unexpectedly cast type customizer already present for %v", pair)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly cast type customizer already present for %v", pair)) } castTypeCustomizers[pair] = customizer alreadyPresent := false @@ -271,7 +272,7 @@ func (c floatCastCustomizer) getCastFunc() castFunc { case types.IntFamily: return floatToInt(c.toWidth, 64) } - colexecerror.InternalError(fmt.Sprintf("unexpectedly didn't find a cast from float to %s with %d width", c.toFamily, c.toWidth)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly didn't find a cast from float to %s with %d width", c.toFamily, c.toWidth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -294,7 +295,7 @@ func (c intCastCustomizer) getCastFunc() castFunc { case types.FloatFamily: return intToFloat() } - colexecerror.InternalError(fmt.Sprintf("unexpectedly didn't find a cast from int to %s with %d width", c.toFamily, c.toWidth)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly didn't find a cast from int to %s with %d width", c.toFamily, c.toWidth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -304,7 +305,7 @@ func (c datumCastCustomizer) getCastFunc() castFunc { case types.BoolFamily: return datumToBool } - colexecerror.InternalError(fmt.Sprintf("unexpectedly didn't find a cast from datum-backed type to %s", c.toFamily)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly didn't find a cast from datum-backed type to %s", c.toFamily)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_gen_util.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_gen_util.go index 74e2ac5ee551..a1105c7859f6 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_gen_util.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_gen_util.go @@ -44,25 +44,25 @@ func populateTwoArgsOverloads( case castOverload: combinableCanonicalTypeFamilies = castableCanonicalTypeFamilies default: - colexecerror.InternalError("unexpectedly overload is neither binary, comparison, nor cast") + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly overload is neither binary, comparison, nor cast")) } for _, leftFamily := range supportedCanonicalTypeFamilies { leftWidths, found := supportedWidthsByCanonicalTypeFamily[leftFamily] if !found { - colexecerror.InternalError(fmt.Sprintf("didn't find supported widths for %s", leftFamily)) + colexecerror.InternalError(errors.AssertionFailedf("didn't find supported widths for %s", leftFamily)) } leftFamilyStr := toString(leftFamily) for _, rightFamily := range combinableCanonicalTypeFamilies[leftFamily] { rightWidths, found := supportedWidthsByCanonicalTypeFamily[rightFamily] if !found { - colexecerror.InternalError(fmt.Sprintf("didn't find supported widths for %s", rightFamily)) + colexecerror.InternalError(errors.AssertionFailedf("didn't find supported widths for %s", rightFamily)) } rightFamilyStr := toString(rightFamily) for _, leftWidth := range leftWidths { for _, rightWidth := range rightWidths { customizer, ok := customizers[typePair{leftFamily, leftWidth, rightFamily, rightWidth}] if !ok { - colexecerror.InternalError("unexpectedly didn't find a type customizer") + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly didn't find a type customizer")) } // Skip overloads that don't have associated output types. retType, ok := opOutputTypes[typePair{leftFamily, leftWidth, rightFamily, rightWidth}] diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_hash.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_hash.go index 33610f794562..bce9f4d610b2 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_hash.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_hash.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) var hashOverloads []*oneArgOverload @@ -28,7 +29,7 @@ func populateHashOverloads() { for _, family := range supportedCanonicalTypeFamilies { widths, found := supportedWidthsByCanonicalTypeFamily[family] if !found { - colexecerror.InternalError(fmt.Sprintf("didn't find supported widths for %s", family)) + colexecerror.InternalError(errors.AssertionFailedf("didn't find supported widths for %s", family)) } ov := newLastArgTypeOverload(hashOverloadBase, family) for _, width := range widths { diff --git a/pkg/sql/colexec/execgen/cmd/execgen/sum_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/sum_agg_gen.go index 2a6c19cd682c..a938547c27e4 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/sum_agg_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/sum_agg_gen.go @@ -11,7 +11,6 @@ package main import ( - "fmt" "io" "strings" "text/template" @@ -19,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) type sumTmplInfo struct { @@ -65,10 +65,10 @@ func getSumAddOverload(inputType *types.T) assignFunc { } } if overload == nil { - colexecerror.InternalError(fmt.Sprintf("unexpectedly didn't find plus binary overload for %s", inputType.String())) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly didn't find plus binary overload for %s", inputType.String())) } if len(overload.WidthOverloads) != 1 { - colexecerror.InternalError(fmt.Sprintf("unexpectedly plus binary overload for %s doesn't contain a single overload", inputType.String())) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly plus binary overload for %s doesn't contain a single overload", inputType.String())) } return overload.WidthOverloads[0].AssignFunc } diff --git a/pkg/sql/colexec/execgen/placeholders.go b/pkg/sql/colexec/execgen/placeholders.go index 0e5dbec6ecb9..ed6612dfad2a 100644 --- a/pkg/sql/colexec/execgen/placeholders.go +++ b/pkg/sql/colexec/execgen/placeholders.go @@ -10,7 +10,10 @@ package execgen -import "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" +import ( + "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" + "github.com/cockroachdb/errors" +) const nonTemplatePanic = "do not call from non-template code" @@ -32,48 +35,48 @@ var ( // source is. You must use this on the result of UNSAFEGET if you wish to store // that result past the lifetime of the batch you UNSAFEGET'd from. func COPYVAL(dest, src interface{}) { - colexecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(errors.AssertionFailedf(nonTemplatePanic)) } // SET is a template function. func SET(target, i, new interface{}) { - colexecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(errors.AssertionFailedf(nonTemplatePanic)) } // SLICE is a template function. func SLICE(target, start, end interface{}) interface{} { - colexecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(errors.AssertionFailedf(nonTemplatePanic)) return nil } // COPYSLICE is a template function. func COPYSLICE(target, src, destIdx, srcStartIdx, srcEndIdx interface{}) { - colexecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(errors.AssertionFailedf(nonTemplatePanic)) } // APPENDSLICE is a template function. func APPENDSLICE(target, src, destIdx, srcStartIdx, srcEndIdx interface{}) { - colexecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(errors.AssertionFailedf(nonTemplatePanic)) } // APPENDVAL is a template function. func APPENDVAL(target, v interface{}) { - colexecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(errors.AssertionFailedf(nonTemplatePanic)) } // LEN is a template function. func LEN(target interface{}) interface{} { - colexecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(errors.AssertionFailedf(nonTemplatePanic)) return nil } // ZERO is a template function. func ZERO(target interface{}) { - colexecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(errors.AssertionFailedf(nonTemplatePanic)) } // WINDOW is a template function. func WINDOW(target, start, end interface{}) interface{} { - colexecerror.InternalError(nonTemplatePanic) + colexecerror.InternalError(errors.AssertionFailedf(nonTemplatePanic)) return nil } diff --git a/pkg/sql/colexec/expr.go b/pkg/sql/colexec/expr.go index fa35a1e4bc24..3851c18e3a51 100644 --- a/pkg/sql/colexec/expr.go +++ b/pkg/sql/colexec/expr.go @@ -11,13 +11,12 @@ package colexec import ( - "fmt" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // ExprHelper is a utility interface that helps with expression handling in @@ -51,7 +50,7 @@ func NewExprHelper(exprDeserialization ExprDeserialization) ExprHelper { case ForcedExprDeserialization: return &forcedDeserializationExprHelper{} default: - colexecerror.InternalError(fmt.Sprintf("unexpected ExprDeserialization %d", exprDeserialization)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected ExprDeserialization %d", exprDeserialization)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/external_hash_joiner.go b/pkg/sql/colexec/external_hash_joiner.go index 9fec8181685c..75b1f28a9d48 100644 --- a/pkg/sql/colexec/external_hash_joiner.go +++ b/pkg/sql/colexec/external_hash_joiner.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "math" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -699,7 +698,7 @@ StateChanged: } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected externalHashJoinerState %d", hj.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected externalHashJoinerState %d", hj.state)) } } } diff --git a/pkg/sql/colexec/external_sort.go b/pkg/sql/colexec/external_sort.go index 892661b02d8f..c136baff3612 100644 --- a/pkg/sql/colexec/external_sort.go +++ b/pkg/sql/colexec/external_sort.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" @@ -358,7 +357,7 @@ func (s *externalSorter) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected externalSorterState %d", s.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected externalSorterState %d", s.state)) } } } diff --git a/pkg/sql/colexec/hash_aggregator.go b/pkg/sql/colexec/hash_aggregator.go index 928028488ccb..f578ebb8040d 100644 --- a/pkg/sql/colexec/hash_aggregator.go +++ b/pkg/sql/colexec/hash_aggregator.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/errors" ) // hashAggregatorState represents the state of the hash aggregator operator. @@ -209,7 +210,7 @@ func (op *hashAggregator) Next(ctx context.Context) coldata.Batch { case hashAggregatorDone: return coldata.ZeroBatch default: - colexecerror.InternalError("hash aggregator in unhandled state") + colexecerror.InternalError(errors.AssertionFailedf("hash aggregator in unhandled state")) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/hash_utils.eg.go b/pkg/sql/colexec/hash_utils.eg.go index 3a532ea4c239..aed4921ab4b9 100644 --- a/pkg/sql/colexec/hash_utils.eg.go +++ b/pkg/sql/colexec/hash_utils.eg.go @@ -11,7 +11,6 @@ package colexec import ( "context" - "fmt" "math" "reflect" "unsafe" @@ -22,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // rehash takes an element of a key (tuple representing a row of equality @@ -908,6 +908,6 @@ func rehash( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", col.Type())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", col.Type())) } } diff --git a/pkg/sql/colexec/hash_utils_tmpl.go b/pkg/sql/colexec/hash_utils_tmpl.go index be669dbbd216..e83182372e39 100644 --- a/pkg/sql/colexec/hash_utils_tmpl.go +++ b/pkg/sql/colexec/hash_utils_tmpl.go @@ -21,12 +21,12 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // {{/* @@ -43,7 +43,7 @@ const _TYPE_WIDTH = 0 // _ASSIGN_HASH is the template equality function for assigning the first input // to the result of the hash value of the second input. func _ASSIGN_HASH(_, _, _, _ interface{}) uint64 { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} @@ -132,6 +132,6 @@ func rehash( } // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", col.Type())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", col.Type())) } } diff --git a/pkg/sql/colexec/hashjoiner.go b/pkg/sql/colexec/hashjoiner.go index 8be02f58424a..f779f8139a4c 100644 --- a/pkg/sql/colexec/hashjoiner.go +++ b/pkg/sql/colexec/hashjoiner.go @@ -299,7 +299,7 @@ func (hj *hashJoiner) Next(ctx context.Context) coldata.Batch { case hjDone: return coldata.ZeroBatch default: - colexecerror.InternalError("hash joiner in unhandled state") + colexecerror.InternalError(errors.AssertionFailedf("hash joiner in unhandled state")) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/hashjoiner_test.go b/pkg/sql/colexec/hashjoiner_test.go index 478ca6e75c4b..c53d651ea4fb 100644 --- a/pkg/sql/colexec/hashjoiner_test.go +++ b/pkg/sql/colexec/hashjoiner_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -44,7 +45,7 @@ func init() { for i, f := range floats { _, err := decs[i].SetFloat64(f) if err != nil { - colexecerror.InternalError(fmt.Sprintf("%v", err)) + colexecerror.InternalError(errors.AssertionFailedf("%v", err)) } } diff --git a/pkg/sql/colexec/hashtable.go b/pkg/sql/colexec/hashtable.go index 2b5139381db3..0ceb67e7554f 100644 --- a/pkg/sql/colexec/hashtable.go +++ b/pkg/sql/colexec/hashtable.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "unsafe" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -21,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // hashTableBuildMode represents different modes in which the hashTable can be @@ -212,7 +212,7 @@ func newHashTable( if !allowNullEquality && probeMode == hashTableDeletingProbeMode { // At the moment, we don't have a use case for such behavior, so let's // assert that it is not requested. - colexecerror.InternalError("hashTableDeletingProbeMode is supported only when null equality is allowed") + colexecerror.InternalError(errors.AssertionFailedf("hashTableDeletingProbeMode is supported only when null equality is allowed")) } // This number was chosen after running benchmarks of all users of the hash // table (hash joiner, hash aggregator, unordered distinct). The reasoning @@ -361,7 +361,7 @@ func (ht *hashTable) build(ctx context.Context, input colexecbase.Operator) { } default: - colexecerror.InternalError(fmt.Sprintf("hashTable in unhandled state")) + colexecerror.InternalError(errors.AssertionFailedf("hashTable in unhandled state")) } } @@ -471,7 +471,7 @@ func (ht *hashTable) checkCols(probeVecs []coldata.Vec, nToCheck uint64, probeSe ht.checkColDeleting(probeVecs[i], ht.vals.ColVec(int(keyCol)), i, nToCheck, probeSel) } default: - colexecerror.InternalError(fmt.Sprintf("unsupported hash table probe mode: %d", ht.probeMode)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported hash table probe mode: %d", ht.probeMode)) } } @@ -590,7 +590,7 @@ func (ht *hashTable) checkBuildForDistinct( probeVecs []coldata.Vec, nToCheck uint64, probeSel []int, ) uint64 { if probeSel == nil { - colexecerror.InternalError("invalid selection vector") + colexecerror.InternalError(errors.AssertionFailedf("invalid selection vector")) } ht.checkColsForDistinctTuples(probeVecs, nToCheck, probeSel) nDiffers := uint64(0) @@ -619,7 +619,7 @@ func (ht *hashTable) checkBuildForAggregation( probeVecs []coldata.Vec, nToCheck uint64, probeSel []int, ) uint64 { if probeSel == nil { - colexecerror.InternalError("invalid selection vector") + colexecerror.InternalError(errors.AssertionFailedf("invalid selection vector")) } ht.checkColsForDistinctTuples(probeVecs, nToCheck, probeSel) nDiffers := uint64(0) diff --git a/pkg/sql/colexec/hashtable_full_deleting.eg.go b/pkg/sql/colexec/hashtable_full_deleting.eg.go index 2033a1414890..33b115f348fb 100644 --- a/pkg/sql/colexec/hashtable_full_deleting.eg.go +++ b/pkg/sql/colexec/hashtable_full_deleting.eg.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // checkColDeleting determines if the current key column in the groupID buckets @@ -8946,7 +8947,7 @@ func (ht *hashTable) check(probeVecs []coldata.Vec, nToCheck uint64, probeSel [] } } default: - colexecerror.InternalError("unsupported hash table probe mode") + colexecerror.InternalError(errors.AssertionFailedf("unsupported hash table probe mode")) } return nDiffers } diff --git a/pkg/sql/colexec/hashtable_tmpl.go b/pkg/sql/colexec/hashtable_tmpl.go index 2669abdf64f1..4ea034bd235a 100644 --- a/pkg/sql/colexec/hashtable_tmpl.go +++ b/pkg/sql/colexec/hashtable_tmpl.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // {{/* @@ -42,7 +43,7 @@ const _RIGHT_TYPE_WIDTH = 0 // _ASSIGN_NE is the template equality function for assigning the first input // to the result of the the second input != the third input. func _ASSIGN_NE(_, _, _, _, _, _ interface{}) int { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // This is a code snippet that is the main body of checkCol* functions. It @@ -459,7 +460,7 @@ func (ht *hashTable) check(probeVecs []coldata.Vec, nToCheck uint64, probeSel [] _CHECK_BODY(false, true) } default: - colexecerror.InternalError("unsupported hash table probe mode") + colexecerror.InternalError(errors.AssertionFailedf("unsupported hash table probe mode")) } return nDiffers } diff --git a/pkg/sql/colexec/materializer.go b/pkg/sql/colexec/materializer.go index e936319d9cd2..2a2207fce498 100644 --- a/pkg/sql/colexec/materializer.go +++ b/pkg/sql/colexec/materializer.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colconv" @@ -22,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // Materializer converts an Operator input into a execinfra.RowSource. @@ -86,7 +86,7 @@ func newDrainHelper(sources execinfrapb.MetadataSources) *drainHelper { // OutputTypes implements the RowSource interface. func (d *drainHelper) OutputTypes() []*types.T { - colexecerror.InternalError("unimplemented") + colexecerror.InternalError(errors.AssertionFailedf("unimplemented")) // Unreachable code. return nil } @@ -196,7 +196,7 @@ func (m *Materializer) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return m.input } - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/mergejoinbase.eg.go b/pkg/sql/colexec/mergejoinbase.eg.go index 6b5e9b992617..cd47dcfa5dbe 100644 --- a/pkg/sql/colexec/mergejoinbase.eg.go +++ b/pkg/sql/colexec/mergejoinbase.eg.go @@ -11,7 +11,6 @@ package colexec import ( "bytes" - "fmt" "math" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -20,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // isBufferedGroupFinished checks to see whether or not the buffered group @@ -476,7 +476,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx])) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx])) } } return false diff --git a/pkg/sql/colexec/mergejoinbase_tmpl.go b/pkg/sql/colexec/mergejoinbase_tmpl.go index 9288e22cc7c0..5818b81fb8d3 100644 --- a/pkg/sql/colexec/mergejoinbase_tmpl.go +++ b/pkg/sql/colexec/mergejoinbase_tmpl.go @@ -20,11 +20,10 @@ package colexec import ( - "fmt" - "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // {{/* @@ -39,7 +38,7 @@ const _TYPE_WIDTH = 0 // _ASSIGN_EQ is the template equality function for assigning the first input // to the result of the the second input == the third input. func _ASSIGN_EQ(_, _, _, _, _, _ interface{}) int { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} @@ -101,7 +100,7 @@ func (o *mergeJoinBase) isBufferedGroupFinished( } // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx])) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx])) } } return false diff --git a/pkg/sql/colexec/mergejoiner_exceptall.eg.go b/pkg/sql/colexec/mergejoiner_exceptall.eg.go index 43548ec4c9e4..0acb67c1ae6d 100644 --- a/pkg/sql/colexec/mergejoiner_exceptall.eg.go +++ b/pkg/sql/colexec/mergejoiner_exceptall.eg.go @@ -12,7 +12,6 @@ package colexec import ( "bytes" "context" - "fmt" "math" "time" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/errors" ) type mergeJoinExceptAllOp struct { @@ -70,7 +70,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -323,7 +323,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -545,7 +545,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -766,7 +766,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1027,7 +1027,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1289,7 +1289,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1554,7 +1554,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1851,7 +1851,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2101,7 +2101,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2323,7 +2323,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2530,7 +2530,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -2558,7 +2558,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2787,7 +2787,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2985,7 +2985,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3182,7 +3182,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3419,7 +3419,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3657,7 +3657,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3898,7 +3898,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4171,7 +4171,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4397,7 +4397,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4595,7 +4595,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4778,7 +4778,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -4808,7 +4808,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5023,7 +5023,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5207,7 +5207,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5390,7 +5390,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5613,7 +5613,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5837,7 +5837,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6064,7 +6064,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6323,7 +6323,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6535,7 +6535,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6719,7 +6719,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6888,7 +6888,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -6916,7 +6916,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7120,7 +7120,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7293,7 +7293,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7465,7 +7465,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7677,7 +7677,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7890,7 +7890,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8106,7 +8106,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8354,7 +8354,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8555,7 +8555,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8728,7 +8728,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8886,7 +8886,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -8933,7 +8933,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9186,7 +9186,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9408,7 +9408,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9629,7 +9629,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9890,7 +9890,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10152,7 +10152,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10417,7 +10417,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10714,7 +10714,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10964,7 +10964,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11186,7 +11186,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11393,7 +11393,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -11421,7 +11421,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11650,7 +11650,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11848,7 +11848,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12045,7 +12045,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12282,7 +12282,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12520,7 +12520,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12761,7 +12761,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13034,7 +13034,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13260,7 +13260,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13458,7 +13458,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13641,7 +13641,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -13671,7 +13671,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13886,7 +13886,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14070,7 +14070,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14253,7 +14253,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14476,7 +14476,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14700,7 +14700,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14927,7 +14927,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15186,7 +15186,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15398,7 +15398,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15582,7 +15582,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15751,7 +15751,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -15779,7 +15779,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15983,7 +15983,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16156,7 +16156,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16328,7 +16328,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16540,7 +16540,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16753,7 +16753,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16969,7 +16969,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17217,7 +17217,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17418,7 +17418,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17591,7 +17591,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17749,7 +17749,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -17796,7 +17796,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18049,7 +18049,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18271,7 +18271,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18492,7 +18492,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18753,7 +18753,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19015,7 +19015,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19280,7 +19280,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19577,7 +19577,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19827,7 +19827,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20049,7 +20049,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20256,7 +20256,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -20284,7 +20284,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20513,7 +20513,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20711,7 +20711,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20908,7 +20908,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21145,7 +21145,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21383,7 +21383,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21624,7 +21624,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21897,7 +21897,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22123,7 +22123,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22321,7 +22321,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22504,7 +22504,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -22534,7 +22534,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22749,7 +22749,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22933,7 +22933,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23116,7 +23116,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23339,7 +23339,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23563,7 +23563,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23790,7 +23790,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24049,7 +24049,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24261,7 +24261,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24445,7 +24445,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24614,7 +24614,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -24642,7 +24642,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24846,7 +24846,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25019,7 +25019,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25191,7 +25191,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25403,7 +25403,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25616,7 +25616,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25832,7 +25832,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26080,7 +26080,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26281,7 +26281,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26454,7 +26454,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26612,7 +26612,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -26659,7 +26659,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26912,7 +26912,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27134,7 +27134,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27355,7 +27355,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27616,7 +27616,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27878,7 +27878,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28143,7 +28143,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28440,7 +28440,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28690,7 +28690,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28912,7 +28912,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29119,7 +29119,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -29147,7 +29147,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29376,7 +29376,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29574,7 +29574,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29771,7 +29771,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30008,7 +30008,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30246,7 +30246,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30487,7 +30487,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30760,7 +30760,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30986,7 +30986,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31184,7 +31184,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31367,7 +31367,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -31397,7 +31397,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31612,7 +31612,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31796,7 +31796,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31979,7 +31979,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32202,7 +32202,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32426,7 +32426,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32653,7 +32653,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32912,7 +32912,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33124,7 +33124,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33308,7 +33308,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33477,7 +33477,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -33505,7 +33505,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33709,7 +33709,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33882,7 +33882,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34054,7 +34054,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34266,7 +34266,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34479,7 +34479,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34695,7 +34695,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34943,7 +34943,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35144,7 +35144,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35317,7 +35317,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35475,7 +35475,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -36171,7 +36171,7 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -36789,7 +36789,7 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -37429,7 +37429,7 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -38037,7 +38037,7 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -38726,7 +38726,7 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } if colIdx == len(input.sourceTypes)-1 { // We have appended some tuples into the output batch from the current @@ -39476,7 +39476,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -40134,7 +40134,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -40814,7 +40814,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -41472,7 +41472,7 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -41810,7 +41810,7 @@ func (o *mergeJoinExceptAllOp) buildRightBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } outStartIdx += toAppend @@ -41970,7 +41970,7 @@ func (o *mergeJoinExceptAllOp) build(ctx context.Context) { o.buildLeftBufferedGroup(ctx, o.builderState.lGroups[0], &o.left, o.proberState.lBufferedGroup, outStartIdx) default: - colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -42057,7 +42057,7 @@ func (o *mergeJoinExceptAllOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/mergejoiner_fullouter.eg.go b/pkg/sql/colexec/mergejoiner_fullouter.eg.go index 9f7e7a40f6b7..323c94533868 100644 --- a/pkg/sql/colexec/mergejoiner_fullouter.eg.go +++ b/pkg/sql/colexec/mergejoiner_fullouter.eg.go @@ -12,7 +12,6 @@ package colexec import ( "bytes" "context" - "fmt" "math" "time" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/errors" ) type mergeJoinFullOuterOp struct { @@ -70,7 +70,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -81,7 +81,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -349,7 +349,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -360,7 +360,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -589,7 +589,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -600,7 +600,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -828,7 +828,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -839,7 +839,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1118,7 +1118,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1129,7 +1129,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1409,7 +1409,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1420,7 +1420,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1703,7 +1703,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1714,7 +1714,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2037,7 +2037,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2048,7 +2048,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2312,7 +2312,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2323,7 +2323,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2552,7 +2552,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2563,7 +2563,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2779,7 +2779,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -2807,7 +2807,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2818,7 +2818,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3072,7 +3072,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3083,7 +3083,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3298,7 +3298,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3309,7 +3309,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3523,7 +3523,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3534,7 +3534,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3799,7 +3799,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3810,7 +3810,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4076,7 +4076,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4087,7 +4087,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4356,7 +4356,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4367,7 +4367,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4676,7 +4676,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4687,7 +4687,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4937,7 +4937,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4948,7 +4948,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5163,7 +5163,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5174,7 +5174,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5376,7 +5376,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -5406,7 +5406,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5417,7 +5417,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5671,7 +5671,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5682,7 +5682,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5897,7 +5897,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5908,7 +5908,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6122,7 +6122,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6133,7 +6133,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6398,7 +6398,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6409,7 +6409,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6675,7 +6675,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6686,7 +6686,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6955,7 +6955,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6966,7 +6966,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7275,7 +7275,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7286,7 +7286,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7536,7 +7536,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7547,7 +7547,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7762,7 +7762,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7773,7 +7773,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7975,7 +7975,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -8003,7 +8003,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8014,7 +8014,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8254,7 +8254,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8265,7 +8265,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8466,7 +8466,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8477,7 +8477,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8677,7 +8677,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8688,7 +8688,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8939,7 +8939,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8950,7 +8950,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9202,7 +9202,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9213,7 +9213,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9468,7 +9468,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9479,7 +9479,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9774,7 +9774,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9785,7 +9785,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10021,7 +10021,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10032,7 +10032,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10233,7 +10233,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10244,7 +10244,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10432,7 +10432,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -10479,7 +10479,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10490,7 +10490,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10758,7 +10758,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10769,7 +10769,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10998,7 +10998,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11009,7 +11009,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11237,7 +11237,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11248,7 +11248,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11527,7 +11527,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11538,7 +11538,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11818,7 +11818,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11829,7 +11829,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12112,7 +12112,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12123,7 +12123,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12446,7 +12446,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12457,7 +12457,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12721,7 +12721,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12732,7 +12732,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12961,7 +12961,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12972,7 +12972,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13188,7 +13188,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -13216,7 +13216,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13227,7 +13227,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13481,7 +13481,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13492,7 +13492,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13707,7 +13707,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13718,7 +13718,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13932,7 +13932,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13943,7 +13943,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14208,7 +14208,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14219,7 +14219,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14485,7 +14485,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14496,7 +14496,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14765,7 +14765,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14776,7 +14776,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15085,7 +15085,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15096,7 +15096,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15346,7 +15346,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15357,7 +15357,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15572,7 +15572,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15583,7 +15583,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15785,7 +15785,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -15815,7 +15815,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15826,7 +15826,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16080,7 +16080,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16091,7 +16091,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16306,7 +16306,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16317,7 +16317,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16531,7 +16531,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16542,7 +16542,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16807,7 +16807,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16818,7 +16818,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17084,7 +17084,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17095,7 +17095,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17364,7 +17364,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17375,7 +17375,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17684,7 +17684,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17695,7 +17695,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17945,7 +17945,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17956,7 +17956,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18171,7 +18171,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18182,7 +18182,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18384,7 +18384,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -18412,7 +18412,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18423,7 +18423,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18663,7 +18663,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18674,7 +18674,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18875,7 +18875,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18886,7 +18886,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19086,7 +19086,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19097,7 +19097,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19348,7 +19348,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19359,7 +19359,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19611,7 +19611,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19622,7 +19622,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19877,7 +19877,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19888,7 +19888,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20183,7 +20183,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20194,7 +20194,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20430,7 +20430,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20441,7 +20441,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20642,7 +20642,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20653,7 +20653,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20841,7 +20841,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -20888,7 +20888,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20899,7 +20899,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21167,7 +21167,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21178,7 +21178,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21407,7 +21407,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21418,7 +21418,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21646,7 +21646,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21657,7 +21657,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21936,7 +21936,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21947,7 +21947,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22227,7 +22227,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22238,7 +22238,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22521,7 +22521,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22532,7 +22532,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22855,7 +22855,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22866,7 +22866,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23130,7 +23130,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23141,7 +23141,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23370,7 +23370,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23381,7 +23381,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23597,7 +23597,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -23625,7 +23625,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23636,7 +23636,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23890,7 +23890,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23901,7 +23901,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24116,7 +24116,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24127,7 +24127,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24341,7 +24341,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24352,7 +24352,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24617,7 +24617,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24628,7 +24628,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24894,7 +24894,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24905,7 +24905,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25174,7 +25174,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25185,7 +25185,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25494,7 +25494,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25505,7 +25505,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25755,7 +25755,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25766,7 +25766,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25981,7 +25981,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25992,7 +25992,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26194,7 +26194,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -26224,7 +26224,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26235,7 +26235,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26489,7 +26489,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26500,7 +26500,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26715,7 +26715,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26726,7 +26726,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26940,7 +26940,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26951,7 +26951,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27216,7 +27216,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27227,7 +27227,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27493,7 +27493,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27504,7 +27504,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27773,7 +27773,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27784,7 +27784,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28093,7 +28093,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28104,7 +28104,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28354,7 +28354,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28365,7 +28365,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28580,7 +28580,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28591,7 +28591,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28793,7 +28793,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -28821,7 +28821,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28832,7 +28832,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29072,7 +29072,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29083,7 +29083,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29284,7 +29284,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29295,7 +29295,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29495,7 +29495,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29506,7 +29506,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29757,7 +29757,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29768,7 +29768,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30020,7 +30020,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30031,7 +30031,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30286,7 +30286,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30297,7 +30297,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30592,7 +30592,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30603,7 +30603,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30839,7 +30839,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30850,7 +30850,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31051,7 +31051,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31062,7 +31062,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31250,7 +31250,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -31297,7 +31297,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31308,7 +31308,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31576,7 +31576,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31587,7 +31587,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31816,7 +31816,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31827,7 +31827,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32055,7 +32055,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32066,7 +32066,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32345,7 +32345,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32356,7 +32356,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32636,7 +32636,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32647,7 +32647,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32930,7 +32930,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32941,7 +32941,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33264,7 +33264,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33275,7 +33275,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33539,7 +33539,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33550,7 +33550,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33779,7 +33779,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33790,7 +33790,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34006,7 +34006,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -34034,7 +34034,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34045,7 +34045,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34299,7 +34299,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34310,7 +34310,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34525,7 +34525,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34536,7 +34536,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34750,7 +34750,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -34761,7 +34761,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35026,7 +35026,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35037,7 +35037,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35303,7 +35303,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35314,7 +35314,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35583,7 +35583,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35594,7 +35594,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35903,7 +35903,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -35914,7 +35914,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -36164,7 +36164,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -36175,7 +36175,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -36390,7 +36390,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -36401,7 +36401,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -36603,7 +36603,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -36633,7 +36633,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -36644,7 +36644,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -36898,7 +36898,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -36909,7 +36909,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -37124,7 +37124,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -37135,7 +37135,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -37349,7 +37349,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -37360,7 +37360,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -37625,7 +37625,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -37636,7 +37636,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -37902,7 +37902,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -37913,7 +37913,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -38182,7 +38182,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -38193,7 +38193,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -38502,7 +38502,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -38513,7 +38513,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -38763,7 +38763,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -38774,7 +38774,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -38989,7 +38989,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -39000,7 +39000,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -39202,7 +39202,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -39230,7 +39230,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -39241,7 +39241,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -39481,7 +39481,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -39492,7 +39492,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -39693,7 +39693,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -39704,7 +39704,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -39904,7 +39904,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -39915,7 +39915,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -40166,7 +40166,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -40177,7 +40177,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -40429,7 +40429,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -40440,7 +40440,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -40695,7 +40695,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -40706,7 +40706,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -41001,7 +41001,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -41012,7 +41012,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -41248,7 +41248,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -41259,7 +41259,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -41460,7 +41460,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -41471,7 +41471,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -41659,7 +41659,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -42355,7 +42355,7 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -42973,7 +42973,7 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -43613,7 +43613,7 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -44221,7 +44221,7 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -44760,7 +44760,7 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } if colIdx == len(input.sourceTypes)-1 { // We have appended some tuples into the output batch from the current @@ -45530,7 +45530,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -46208,7 +46208,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -46908,7 +46908,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -47586,7 +47586,7 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -47924,7 +47924,7 @@ func (o *mergeJoinFullOuterOp) buildRightBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } outStartIdx += toAppend @@ -48105,7 +48105,7 @@ func (o *mergeJoinFullOuterOp) build(ctx context.Context) { o.buildRightBufferedGroup(ctx, o.builderState.rGroups[0], len(o.left.sourceTypes), &o.right, o.proberState.rBufferedGroup, outStartIdx) default: - colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -48203,7 +48203,7 @@ func (o *mergeJoinFullOuterOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/mergejoiner_inner.eg.go b/pkg/sql/colexec/mergejoiner_inner.eg.go index 54df5f9f0827..7b103bea8e59 100644 --- a/pkg/sql/colexec/mergejoiner_inner.eg.go +++ b/pkg/sql/colexec/mergejoiner_inner.eg.go @@ -12,7 +12,6 @@ package colexec import ( "bytes" "context" - "fmt" "math" "time" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/errors" ) type mergeJoinInnerOp struct { @@ -1761,7 +1761,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -3380,7 +3380,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -5001,7 +5001,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -6520,7 +6520,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -8258,7 +8258,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -9877,7 +9877,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -11498,7 +11498,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -13017,7 +13017,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -14755,7 +14755,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -16374,7 +16374,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -17995,7 +17995,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -19514,7 +19514,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -21252,7 +21252,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -22871,7 +22871,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -24492,7 +24492,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -26011,7 +26011,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -26677,7 +26677,7 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -27265,7 +27265,7 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -27875,7 +27875,7 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -28453,7 +28453,7 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -28992,7 +28992,7 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } if colIdx == len(input.sourceTypes)-1 { // We have appended some tuples into the output batch from the current @@ -29742,7 +29742,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -30400,7 +30400,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -31080,7 +31080,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -31738,7 +31738,7 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -32076,7 +32076,7 @@ func (o *mergeJoinInnerOp) buildRightBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } outStartIdx += toAppend @@ -32219,7 +32219,7 @@ func (o *mergeJoinInnerOp) build(ctx context.Context) { o.buildRightBufferedGroup(ctx, o.builderState.rGroups[0], len(o.left.sourceTypes), &o.right, o.proberState.rBufferedGroup, outStartIdx) default: - colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -32290,7 +32290,7 @@ func (o *mergeJoinInnerOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/mergejoiner_intersectall.eg.go b/pkg/sql/colexec/mergejoiner_intersectall.eg.go index d55a8685813b..7b1804a06515 100644 --- a/pkg/sql/colexec/mergejoiner_intersectall.eg.go +++ b/pkg/sql/colexec/mergejoiner_intersectall.eg.go @@ -12,7 +12,6 @@ package colexec import ( "bytes" "context" - "fmt" "math" "time" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/errors" ) type mergeJoinIntersectAllOp struct { @@ -1941,7 +1941,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -3600,7 +3600,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -5241,7 +5241,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -6770,7 +6770,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -8688,7 +8688,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -10347,7 +10347,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -11988,7 +11988,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -13517,7 +13517,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -15435,7 +15435,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -17094,7 +17094,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -18735,7 +18735,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -20264,7 +20264,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -22182,7 +22182,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -23841,7 +23841,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -25482,7 +25482,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -27011,7 +27011,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -27677,7 +27677,7 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -28265,7 +28265,7 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -28875,7 +28875,7 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -29453,7 +29453,7 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -30142,7 +30142,7 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } if colIdx == len(input.sourceTypes)-1 { // We have appended some tuples into the output batch from the current @@ -30892,7 +30892,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -31550,7 +31550,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -32230,7 +32230,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -32888,7 +32888,7 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -33226,7 +33226,7 @@ func (o *mergeJoinIntersectAllOp) buildRightBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } outStartIdx += toAppend @@ -33367,7 +33367,7 @@ func (o *mergeJoinIntersectAllOp) build(ctx context.Context) { o.buildLeftBufferedGroup(ctx, o.builderState.lGroups[0], &o.left, o.proberState.lBufferedGroup, outStartIdx) default: - colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -33438,7 +33438,7 @@ func (o *mergeJoinIntersectAllOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/mergejoiner_leftanti.eg.go b/pkg/sql/colexec/mergejoiner_leftanti.eg.go index 2cb106a8378b..05e4645db4e5 100644 --- a/pkg/sql/colexec/mergejoiner_leftanti.eg.go +++ b/pkg/sql/colexec/mergejoiner_leftanti.eg.go @@ -12,7 +12,6 @@ package colexec import ( "bytes" "context" - "fmt" "math" "time" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/errors" ) type mergeJoinLeftAntiOp struct { @@ -70,7 +70,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -297,7 +297,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -493,7 +493,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -688,7 +688,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -923,7 +923,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1159,7 +1159,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1398,7 +1398,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1669,7 +1669,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1893,7 +1893,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2089,7 +2089,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2270,7 +2270,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -2298,7 +2298,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2515,7 +2515,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2701,7 +2701,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2886,7 +2886,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3111,7 +3111,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3337,7 +3337,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3566,7 +3566,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3827,7 +3827,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4041,7 +4041,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4227,7 +4227,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4398,7 +4398,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -4428,7 +4428,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4641,7 +4641,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4823,7 +4823,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5004,7 +5004,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5225,7 +5225,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5447,7 +5447,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5672,7 +5672,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5929,7 +5929,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6139,7 +6139,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6321,7 +6321,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6488,7 +6488,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -6516,7 +6516,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6719,7 +6719,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6891,7 +6891,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7062,7 +7062,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7273,7 +7273,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7485,7 +7485,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7700,7 +7700,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7947,7 +7947,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8147,7 +8147,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8319,7 +8319,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8476,7 +8476,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -8523,7 +8523,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8750,7 +8750,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8946,7 +8946,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9141,7 +9141,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9376,7 +9376,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9612,7 +9612,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9851,7 +9851,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10122,7 +10122,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10346,7 +10346,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10542,7 +10542,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10723,7 +10723,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -10751,7 +10751,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10968,7 +10968,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11154,7 +11154,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11339,7 +11339,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11564,7 +11564,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11790,7 +11790,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12019,7 +12019,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12280,7 +12280,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12494,7 +12494,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12680,7 +12680,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12851,7 +12851,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -12881,7 +12881,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13094,7 +13094,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13276,7 +13276,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13457,7 +13457,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13678,7 +13678,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13900,7 +13900,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14125,7 +14125,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14382,7 +14382,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14592,7 +14592,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14774,7 +14774,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14941,7 +14941,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -14969,7 +14969,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15172,7 +15172,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15344,7 +15344,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15515,7 +15515,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15726,7 +15726,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15938,7 +15938,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16153,7 +16153,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16400,7 +16400,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16600,7 +16600,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16772,7 +16772,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16929,7 +16929,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -16976,7 +16976,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17203,7 +17203,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17399,7 +17399,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17594,7 +17594,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17829,7 +17829,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18065,7 +18065,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18304,7 +18304,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18575,7 +18575,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18799,7 +18799,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18995,7 +18995,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19176,7 +19176,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -19204,7 +19204,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19421,7 +19421,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19607,7 +19607,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19792,7 +19792,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20017,7 +20017,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20243,7 +20243,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20472,7 +20472,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20733,7 +20733,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20947,7 +20947,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21133,7 +21133,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21304,7 +21304,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -21334,7 +21334,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21547,7 +21547,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21729,7 +21729,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21910,7 +21910,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22131,7 +22131,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22353,7 +22353,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22578,7 +22578,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22835,7 +22835,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23045,7 +23045,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23227,7 +23227,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23394,7 +23394,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -23422,7 +23422,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23625,7 +23625,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23797,7 +23797,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23968,7 +23968,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24179,7 +24179,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24391,7 +24391,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24606,7 +24606,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24853,7 +24853,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25053,7 +25053,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25225,7 +25225,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25382,7 +25382,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -25429,7 +25429,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25656,7 +25656,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25852,7 +25852,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26047,7 +26047,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26282,7 +26282,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26518,7 +26518,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26757,7 +26757,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27028,7 +27028,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27252,7 +27252,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27448,7 +27448,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27629,7 +27629,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -27657,7 +27657,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27874,7 +27874,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28060,7 +28060,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28245,7 +28245,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28470,7 +28470,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28696,7 +28696,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28925,7 +28925,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29186,7 +29186,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29400,7 +29400,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29586,7 +29586,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29757,7 +29757,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -29787,7 +29787,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30000,7 +30000,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30182,7 +30182,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30363,7 +30363,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30584,7 +30584,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30806,7 +30806,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31031,7 +31031,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31288,7 +31288,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31498,7 +31498,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31680,7 +31680,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31847,7 +31847,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -31875,7 +31875,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32078,7 +32078,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32250,7 +32250,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32421,7 +32421,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32632,7 +32632,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32844,7 +32844,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33059,7 +33059,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33306,7 +33306,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33506,7 +33506,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33678,7 +33678,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33835,7 +33835,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -34531,7 +34531,7 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -35149,7 +35149,7 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -35789,7 +35789,7 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -36397,7 +36397,7 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -36936,7 +36936,7 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } if colIdx == len(input.sourceTypes)-1 { // We have appended some tuples into the output batch from the current @@ -37686,7 +37686,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -38344,7 +38344,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -39024,7 +39024,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -39682,7 +39682,7 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -40020,7 +40020,7 @@ func (o *mergeJoinLeftAntiOp) buildRightBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } outStartIdx += toAppend @@ -40164,7 +40164,7 @@ func (o *mergeJoinLeftAntiOp) build(ctx context.Context) { o.buildLeftBufferedGroup(ctx, o.builderState.lGroups[0], &o.left, o.proberState.lBufferedGroup, outStartIdx) default: - colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -40251,7 +40251,7 @@ func (o *mergeJoinLeftAntiOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/mergejoiner_leftouter.eg.go b/pkg/sql/colexec/mergejoiner_leftouter.eg.go index e0c74eef377c..d14b1f582fe1 100644 --- a/pkg/sql/colexec/mergejoiner_leftouter.eg.go +++ b/pkg/sql/colexec/mergejoiner_leftouter.eg.go @@ -12,7 +12,6 @@ package colexec import ( "bytes" "context" - "fmt" "math" "time" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/errors" ) type mergeJoinLeftOuterOp struct { @@ -70,7 +70,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -298,7 +298,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -495,7 +495,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -691,7 +691,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -927,7 +927,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1164,7 +1164,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1404,7 +1404,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1676,7 +1676,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1901,7 +1901,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2098,7 +2098,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2280,7 +2280,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -2308,7 +2308,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2526,7 +2526,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2713,7 +2713,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2899,7 +2899,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3125,7 +3125,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3352,7 +3352,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3582,7 +3582,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3844,7 +3844,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4059,7 +4059,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4246,7 +4246,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4418,7 +4418,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -4448,7 +4448,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4662,7 +4662,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4845,7 +4845,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5027,7 +5027,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5249,7 +5249,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5472,7 +5472,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5698,7 +5698,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5956,7 +5956,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6167,7 +6167,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6350,7 +6350,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6518,7 +6518,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -6546,7 +6546,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6750,7 +6750,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6923,7 +6923,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7095,7 +7095,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7307,7 +7307,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7520,7 +7520,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7736,7 +7736,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7984,7 +7984,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8185,7 +8185,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8358,7 +8358,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8516,7 +8516,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -8563,7 +8563,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8791,7 +8791,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8988,7 +8988,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9184,7 +9184,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9420,7 +9420,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9657,7 +9657,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9897,7 +9897,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10169,7 +10169,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10394,7 +10394,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10591,7 +10591,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10773,7 +10773,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -10801,7 +10801,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11019,7 +11019,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11206,7 +11206,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11392,7 +11392,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11618,7 +11618,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11845,7 +11845,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12075,7 +12075,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12337,7 +12337,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12552,7 +12552,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12739,7 +12739,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12911,7 +12911,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -12941,7 +12941,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13155,7 +13155,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13338,7 +13338,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13520,7 +13520,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13742,7 +13742,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13965,7 +13965,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14191,7 +14191,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14449,7 +14449,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14660,7 +14660,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14843,7 +14843,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15011,7 +15011,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -15039,7 +15039,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15243,7 +15243,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15416,7 +15416,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15588,7 +15588,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15800,7 +15800,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16013,7 +16013,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16229,7 +16229,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16477,7 +16477,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16678,7 +16678,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16851,7 +16851,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17009,7 +17009,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -17056,7 +17056,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17284,7 +17284,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17481,7 +17481,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17677,7 +17677,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17913,7 +17913,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18150,7 +18150,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18390,7 +18390,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18662,7 +18662,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18887,7 +18887,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19084,7 +19084,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19266,7 +19266,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -19294,7 +19294,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19512,7 +19512,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19699,7 +19699,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19885,7 +19885,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20111,7 +20111,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20338,7 +20338,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20568,7 +20568,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20830,7 +20830,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21045,7 +21045,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21232,7 +21232,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21404,7 +21404,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -21434,7 +21434,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21648,7 +21648,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21831,7 +21831,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22013,7 +22013,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22235,7 +22235,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22458,7 +22458,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22684,7 +22684,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22942,7 +22942,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23153,7 +23153,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23336,7 +23336,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23504,7 +23504,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -23532,7 +23532,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23736,7 +23736,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23909,7 +23909,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24081,7 +24081,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24293,7 +24293,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24506,7 +24506,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24722,7 +24722,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24970,7 +24970,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25171,7 +25171,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25344,7 +25344,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25502,7 +25502,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -25549,7 +25549,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25777,7 +25777,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25974,7 +25974,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26170,7 +26170,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26406,7 +26406,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26643,7 +26643,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26883,7 +26883,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27155,7 +27155,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27380,7 +27380,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27577,7 +27577,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27759,7 +27759,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -27787,7 +27787,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28005,7 +28005,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28192,7 +28192,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28378,7 +28378,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28604,7 +28604,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28831,7 +28831,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29061,7 +29061,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29323,7 +29323,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29538,7 +29538,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29725,7 +29725,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29897,7 +29897,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -29927,7 +29927,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30141,7 +30141,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30324,7 +30324,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30506,7 +30506,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30728,7 +30728,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30951,7 +30951,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31177,7 +31177,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31435,7 +31435,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31646,7 +31646,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31829,7 +31829,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31997,7 +31997,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -32025,7 +32025,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32229,7 +32229,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32402,7 +32402,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32574,7 +32574,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32786,7 +32786,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32999,7 +32999,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33215,7 +33215,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33463,7 +33463,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33664,7 +33664,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33837,7 +33837,7 @@ EqLoop: if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33995,7 +33995,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -34661,7 +34661,7 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -35249,7 +35249,7 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -35859,7 +35859,7 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -36437,7 +36437,7 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -36976,7 +36976,7 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } if colIdx == len(input.sourceTypes)-1 { // We have appended some tuples into the output batch from the current @@ -37746,7 +37746,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -38424,7 +38424,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -39124,7 +39124,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -39802,7 +39802,7 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -40140,7 +40140,7 @@ func (o *mergeJoinLeftOuterOp) buildRightBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } outStartIdx += toAppend @@ -40303,7 +40303,7 @@ func (o *mergeJoinLeftOuterOp) build(ctx context.Context) { o.buildRightBufferedGroup(ctx, o.builderState.rGroups[0], len(o.left.sourceTypes), &o.right, o.proberState.rBufferedGroup, outStartIdx) default: - colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -40390,7 +40390,7 @@ func (o *mergeJoinLeftOuterOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/mergejoiner_leftsemi.eg.go b/pkg/sql/colexec/mergejoiner_leftsemi.eg.go index cb06770f1f76..94d0b788d18c 100644 --- a/pkg/sql/colexec/mergejoiner_leftsemi.eg.go +++ b/pkg/sql/colexec/mergejoiner_leftsemi.eg.go @@ -12,7 +12,6 @@ package colexec import ( "bytes" "context" - "fmt" "math" "time" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/errors" ) type mergeJoinLeftSemiOp struct { @@ -1751,7 +1751,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -3360,7 +3360,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -4971,7 +4971,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -6480,7 +6480,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -8208,7 +8208,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -9817,7 +9817,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -11428,7 +11428,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -12937,7 +12937,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -14665,7 +14665,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -16274,7 +16274,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -17885,7 +17885,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -19394,7 +19394,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -21122,7 +21122,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -22731,7 +22731,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -24342,7 +24342,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -25851,7 +25851,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -26517,7 +26517,7 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -27105,7 +27105,7 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -27715,7 +27715,7 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -28293,7 +28293,7 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -28832,7 +28832,7 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } if colIdx == len(input.sourceTypes)-1 { // We have appended some tuples into the output batch from the current @@ -29582,7 +29582,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -30240,7 +30240,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -30920,7 +30920,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -31578,7 +31578,7 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -31916,7 +31916,7 @@ func (o *mergeJoinLeftSemiOp) buildRightBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } outStartIdx += toAppend @@ -32054,7 +32054,7 @@ func (o *mergeJoinLeftSemiOp) build(ctx context.Context) { o.buildLeftBufferedGroup(ctx, o.builderState.lGroups[0], &o.left, o.proberState.lBufferedGroup, outStartIdx) default: - colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -32125,7 +32125,7 @@ func (o *mergeJoinLeftSemiOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/mergejoiner_rightouter.eg.go b/pkg/sql/colexec/mergejoiner_rightouter.eg.go index 47fe21825961..1e815de781e7 100644 --- a/pkg/sql/colexec/mergejoiner_rightouter.eg.go +++ b/pkg/sql/colexec/mergejoiner_rightouter.eg.go @@ -12,7 +12,6 @@ package colexec import ( "bytes" "context" - "fmt" "math" "time" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/errors" ) type mergeJoinRightOuterOp struct { @@ -70,7 +70,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -296,7 +296,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -491,7 +491,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -685,7 +685,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -919,7 +919,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1154,7 +1154,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1392,7 +1392,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1662,7 +1662,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -1885,7 +1885,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2080,7 +2080,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2260,7 +2260,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -2288,7 +2288,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2500,7 +2500,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2681,7 +2681,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -2861,7 +2861,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3081,7 +3081,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3302,7 +3302,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3526,7 +3526,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3782,7 +3782,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -3991,7 +3991,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4172,7 +4172,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4338,7 +4338,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -4368,7 +4368,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4584,7 +4584,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4769,7 +4769,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -4953,7 +4953,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5177,7 +5177,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5402,7 +5402,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5630,7 +5630,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -5890,7 +5890,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6103,7 +6103,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6288,7 +6288,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6458,7 +6458,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -6486,7 +6486,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6688,7 +6688,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -6859,7 +6859,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7029,7 +7029,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7239,7 +7239,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7450,7 +7450,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7664,7 +7664,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -7910,7 +7910,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8109,7 +8109,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8280,7 +8280,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8436,7 +8436,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -8483,7 +8483,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8709,7 +8709,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -8904,7 +8904,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9098,7 +9098,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9332,7 +9332,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9567,7 +9567,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -9805,7 +9805,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10075,7 +10075,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10298,7 +10298,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10493,7 +10493,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10673,7 +10673,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -10701,7 +10701,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -10913,7 +10913,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11094,7 +11094,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11274,7 +11274,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11494,7 +11494,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11715,7 +11715,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -11939,7 +11939,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12195,7 +12195,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12404,7 +12404,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12585,7 +12585,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12751,7 +12751,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -12781,7 +12781,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -12997,7 +12997,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13182,7 +13182,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13366,7 +13366,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13590,7 +13590,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -13815,7 +13815,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14043,7 +14043,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14303,7 +14303,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14516,7 +14516,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14701,7 +14701,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -14871,7 +14871,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -14899,7 +14899,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15101,7 +15101,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15272,7 +15272,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15442,7 +15442,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15652,7 +15652,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -15863,7 +15863,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16077,7 +16077,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16323,7 +16323,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16522,7 +16522,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16693,7 +16693,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -16849,7 +16849,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -16896,7 +16896,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17122,7 +17122,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17317,7 +17317,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17511,7 +17511,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17745,7 +17745,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -17980,7 +17980,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18218,7 +18218,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18488,7 +18488,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18711,7 +18711,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -18906,7 +18906,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19086,7 +19086,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -19114,7 +19114,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19326,7 +19326,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19507,7 +19507,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19687,7 +19687,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -19907,7 +19907,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20128,7 +20128,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20352,7 +20352,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20608,7 +20608,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20817,7 +20817,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -20998,7 +20998,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21164,7 +21164,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -21194,7 +21194,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21410,7 +21410,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21595,7 +21595,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -21779,7 +21779,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22003,7 +22003,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22228,7 +22228,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22456,7 +22456,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22716,7 +22716,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -22929,7 +22929,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23114,7 +23114,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23284,7 +23284,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -23312,7 +23312,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23514,7 +23514,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23685,7 +23685,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -23855,7 +23855,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24065,7 +24065,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24276,7 +24276,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24490,7 +24490,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24736,7 +24736,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -24935,7 +24935,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25106,7 +25106,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25262,7 +25262,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -25309,7 +25309,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25535,7 +25535,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25730,7 +25730,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -25924,7 +25924,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26158,7 +26158,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26393,7 +26393,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26631,7 +26631,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -26901,7 +26901,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27124,7 +27124,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27319,7 +27319,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27499,7 +27499,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -27527,7 +27527,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27739,7 +27739,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -27920,7 +27920,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28100,7 +28100,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28320,7 +28320,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28541,7 +28541,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -28765,7 +28765,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29021,7 +29021,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29230,7 +29230,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29411,7 +29411,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29577,7 +29577,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } else { @@ -29607,7 +29607,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -29823,7 +29823,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30008,7 +30008,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30192,7 +30192,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30416,7 +30416,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30641,7 +30641,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -30869,7 +30869,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31129,7 +31129,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31342,7 +31342,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31527,7 +31527,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31697,7 +31697,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } else { @@ -31725,7 +31725,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -31927,7 +31927,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32098,7 +32098,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32268,7 +32268,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32478,7 +32478,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32689,7 +32689,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -32903,7 +32903,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33149,7 +33149,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33348,7 +33348,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33519,7 +33519,7 @@ EqLoop: if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -33675,7 +33675,7 @@ EqLoop: } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } } } @@ -34371,7 +34371,7 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -34989,7 +34989,7 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -35629,7 +35629,7 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -36237,7 +36237,7 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( o.builderState.left.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -36776,7 +36776,7 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } if colIdx == len(input.sourceTypes)-1 { // We have appended some tuples into the output batch from the current @@ -37526,7 +37526,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -38184,7 +38184,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } else { @@ -38864,7 +38864,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } else { @@ -39522,7 +39522,7 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.groupsIdx = zeroMJCPGroupsIdx } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } } @@ -39860,7 +39860,7 @@ func (o *mergeJoinRightOuterOp) buildRightBufferedGroup( } } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } outStartIdx += toAppend @@ -40021,7 +40021,7 @@ func (o *mergeJoinRightOuterOp) build(ctx context.Context) { o.buildRightBufferedGroup(ctx, o.builderState.rGroups[0], len(o.left.sourceTypes), &o.right, o.proberState.rBufferedGroup, outStartIdx) default: - colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -40108,7 +40108,7 @@ func (o *mergeJoinRightOuterOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/mergejoiner_tmpl.go b/pkg/sql/colexec/mergejoiner_tmpl.go index d880c1089fc9..dbce11a18744 100644 --- a/pkg/sql/colexec/mergejoiner_tmpl.go +++ b/pkg/sql/colexec/mergejoiner_tmpl.go @@ -21,13 +21,13 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // {{/* @@ -45,7 +45,7 @@ const _TYPE_WIDTH = 0 // _ASSIGN_EQ is the template equality function for assigning the first input // to the result of the second input == the third input. func _ASSIGN_EQ(_, _, _, _, _, _ interface{}) int { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // _ASSIGN_CMP is the template equality function for assigning the first input @@ -370,7 +370,7 @@ func _PROBE_SWITCH( } // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", colType)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", colType)) } // {{end}} // {{/* @@ -392,7 +392,7 @@ func _LEFT_UNMATCHED_GROUP_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // {{if or $.JoinType.IsLeftOuter $.JoinType.IsLeftAnti}} if lGroup.unmatched { if curLIdx+1 != curLEndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the left unmatched group is not 1", curLEndIdx-curLIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -434,7 +434,7 @@ func _RIGHT_UNMATCHED_GROUP_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // {{if $.JoinType.IsRightOuter}} if rGroup.unmatched { if curRIdx+1 != curREndIdx { - colexecerror.InternalError(fmt.Sprintf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly length %d of the right unmatched group is not 1", curREndIdx-curRIdx)) } // The row already does not have a match, so we don't need to do any // additional processing. @@ -798,7 +798,7 @@ func _LEFT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool) } // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } // {{end}} // {{/* @@ -1001,7 +1001,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( } // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } if colIdx == len(input.sourceTypes)-1 { // We have appended some tuples into the output batch from the current @@ -1141,7 +1141,7 @@ func _RIGHT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool } // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } // {{end}} // {{/* @@ -1283,7 +1283,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( } // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", input.sourceTypes[colIdx].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", input.sourceTypes[colIdx].String())) } } outStartIdx += toAppend @@ -1538,7 +1538,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) build(ctx context.Context) { // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) + colexecerror.InternalError(errors.AssertionFailedf("unsupported mjBuildFrom %d", o.builderState.buildFrom)) } } } @@ -1656,7 +1656,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) Next(ctx context.Context) coldata.Batch { } return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("unexpected merge joiner state in Next: %v", o.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected merge joiner state in Next: %v", o.state)) } } } diff --git a/pkg/sql/colexec/min_max_agg_tmpl.go b/pkg/sql/colexec/min_max_agg_tmpl.go index 12f42d5fd139..54cc4c17da21 100644 --- a/pkg/sql/colexec/min_max_agg_tmpl.go +++ b/pkg/sql/colexec/min_max_agg_tmpl.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // Remove unused warning. @@ -40,13 +41,13 @@ var _ = colexecerror.InternalError // if the second input compares successfully to the third input. The comparison // operator is tree.LT for MIN and is tree.GT for MAX. func _ASSIGN_CMP(_, _, _, _, _, _ string) bool { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // _COPYVAL_MAYBE_CAST is the template function for copying the second argument // into the first one, possibly performing a cast in the process. func _COPYVAL_MAYBE_CAST(_, _ string) bool { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/operator.go b/pkg/sql/colexec/operator.go index 990ff35c3b12..6806e6152e1c 100644 --- a/pkg/sql/colexec/operator.go +++ b/pkg/sql/colexec/operator.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -21,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" ) // OperatorInitStatus indicates whether Init method has already been called on @@ -62,7 +62,7 @@ func (n OneInputNode) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return n.input } - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -93,7 +93,7 @@ func (n *twoInputNode) Child(nth int, verbose bool) execinfra.OpNode { case 1: return n.inputTwo } - colexecerror.InternalError(fmt.Sprintf("invalid idx %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid idx %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -449,7 +449,7 @@ func NewBatchSchemaSubsetEnforcer( func (e *BatchSchemaSubsetEnforcer) Init() { e.input.Init() if e.subsetStartIdx >= e.subsetEndIdx { - colexecerror.InternalError("unexpectedly subsetStartIdx is not less than subsetEndIdx") + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly subsetStartIdx is not less than subsetEndIdx")) } } diff --git a/pkg/sql/colexec/ordered_synchronizer.eg.go b/pkg/sql/colexec/ordered_synchronizer.eg.go index 6197def28192..c5e2ebade90d 100644 --- a/pkg/sql/colexec/ordered_synchronizer.eg.go +++ b/pkg/sql/colexec/ordered_synchronizer.eg.go @@ -12,7 +12,6 @@ package colexec import ( "container/heap" "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" @@ -24,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/errors" ) // OrderedSynchronizer receives rows from multiple inputs and produces a single @@ -226,7 +226,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { outCol.Set(outputIdx, v) } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", o.typs[i].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", o.typs[i].String())) } } } @@ -333,7 +333,7 @@ func (o *OrderedSynchronizer) resetOutput() { o.outDatumCols = append(o.outDatumCols, outVec.Datum()) } default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", o.typs[i])) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", o.typs[i])) } } } @@ -390,7 +390,7 @@ func (o *OrderedSynchronizer) compareRow(batchIdx1 int, batchIdx2 int) int { case encoding.Descending: return -res default: - colexecerror.InternalError(fmt.Sprintf("unexpected direction value %d", d)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected direction value %d", d)) } } } diff --git a/pkg/sql/colexec/ordered_synchronizer_tmpl.go b/pkg/sql/colexec/ordered_synchronizer_tmpl.go index 796106f42908..677c7540df5f 100644 --- a/pkg/sql/colexec/ordered_synchronizer_tmpl.go +++ b/pkg/sql/colexec/ordered_synchronizer_tmpl.go @@ -22,7 +22,6 @@ package colexec import ( "container/heap" "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" @@ -35,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/errors" ) // {{/* @@ -176,7 +176,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { } // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", o.typs[i].String())) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", o.typs[i].String())) } } } @@ -226,7 +226,7 @@ func (o *OrderedSynchronizer) resetOutput() { } // {{end}} default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", o.typs[i])) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", o.typs[i])) } } } @@ -283,7 +283,7 @@ func (o *OrderedSynchronizer) compareRow(batchIdx1 int, batchIdx2 int) int { case encoding.Descending: return -res default: - colexecerror.InternalError(fmt.Sprintf("unexpected direction value %d", d)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected direction value %d", d)) } } } diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go index b3f20ca638bc..2528e56d51ad 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go @@ -152,7 +152,7 @@ func TestUnorderedSynchronizerNoLeaksOnError(t *testing.T) { inputs := make([]SynchronizerInput, 6) inputs[0].Op = &colexecbase.CallbackOperator{NextCb: func(context.Context) coldata.Batch { - colexecerror.InternalError(expectedErr) + colexecerror.InternalError(errors.New(expectedErr)) // This code is unreachable, but the compiler cannot infer that. return nil }} diff --git a/pkg/sql/colexec/partially_ordered_distinct.go b/pkg/sql/colexec/partially_ordered_distinct.go index 43adc65d096d..2e3cba19a409 100644 --- a/pkg/sql/colexec/partially_ordered_distinct.go +++ b/pkg/sql/colexec/partially_ordered_distinct.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -86,7 +85,7 @@ func (p *partiallyOrderedDistinct) Child(nth int, _ bool) execinfra.OpNode { if nth == 0 { return p.input } - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -163,7 +162,7 @@ func (c *chunkerOperator) Child(nth int, _ bool) execinfra.OpNode { if nth == 0 { return c.input } - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/proj_const_ops_tmpl.go b/pkg/sql/colexec/proj_const_ops_tmpl.go index b6e438715f4b..5b8ca90ff577 100644 --- a/pkg/sql/colexec/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/proj_const_ops_tmpl.go @@ -56,7 +56,7 @@ type _NON_CONST_GOTYPESLICE interface{} // _ASSIGN is the template function for assigning the first input to the result // of computation an operation on the second and the third inputs. func _ASSIGN(_, _, _, _, _, _ interface{}) { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/proj_non_const_ops_tmpl.go index 906355d10cc3..e33a1da6194b 100644 --- a/pkg/sql/colexec/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/proj_non_const_ops_tmpl.go @@ -52,7 +52,7 @@ const _RIGHT_TYPE_WIDTH = 0 // _ASSIGN is the template function for assigning the first input to the result // of computation an operation on the second and the third inputs. func _ASSIGN(_, _, _, _, _, _ interface{}) { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/rank_tmpl.go b/pkg/sql/colexec/rank_tmpl.go index ffeccf0bc8e6..6062d9ce9ffc 100644 --- a/pkg/sql/colexec/rank_tmpl.go +++ b/pkg/sql/colexec/rank_tmpl.go @@ -82,13 +82,13 @@ func NewRankOperator( // _UPDATE_RANK is the template function for updating the state of rank // operators. func _UPDATE_RANK() { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // _UPDATE_RANK_INCREMENT is the template function for updating the state of // rank operators. func _UPDATE_RANK_INCREMENT() { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/relative_rank.eg.go b/pkg/sql/colexec/relative_rank.eg.go index cefbfa9e3138..bab397a57c6e 100644 --- a/pkg/sql/colexec/relative_rank.eg.go +++ b/pkg/sql/colexec/relative_rank.eg.go @@ -327,7 +327,7 @@ func (r *percentRankNoPartitionOp) Next(ctx context.Context) coldata.Batch { return coldata.ZeroBatch default: - colexecerror.InternalError("percent rank operator in unhandled state") + colexecerror.InternalError(errors.AssertionFailedf("percent rank operator in unhandled state")) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -632,7 +632,7 @@ func (r *percentRankWithPartitionOp) Next(ctx context.Context) coldata.Batch { return coldata.ZeroBatch default: - colexecerror.InternalError("percent rank operator in unhandled state") + colexecerror.InternalError(errors.AssertionFailedf("percent rank operator in unhandled state")) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -928,7 +928,7 @@ func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { return coldata.ZeroBatch default: - colexecerror.InternalError("percent rank operator in unhandled state") + colexecerror.InternalError(errors.AssertionFailedf("percent rank operator in unhandled state")) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -1336,7 +1336,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { return coldata.ZeroBatch default: - colexecerror.InternalError("percent rank operator in unhandled state") + colexecerror.InternalError(errors.AssertionFailedf("percent rank operator in unhandled state")) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/relative_rank_tmpl.go b/pkg/sql/colexec/relative_rank_tmpl.go index d477ad1ae09b..8792344be3e9 100644 --- a/pkg/sql/colexec/relative_rank_tmpl.go +++ b/pkg/sql/colexec/relative_rank_tmpl.go @@ -602,7 +602,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { return coldata.ZeroBatch default: - colexecerror.InternalError("percent rank operator in unhandled state") + colexecerror.InternalError(errors.AssertionFailedf("percent rank operator in unhandled state")) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/routers.go b/pkg/sql/colexec/routers.go index a6b46945a03c..affd830fc2ce 100644 --- a/pkg/sql/colexec/routers.go +++ b/pkg/sql/colexec/routers.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "sync" "sync/atomic" @@ -164,7 +163,7 @@ func (o *routerOutputOp) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return o.input } - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/select_in_tmpl.go b/pkg/sql/colexec/select_in_tmpl.go index 0aac4886eaec..89e41ee52fdc 100644 --- a/pkg/sql/colexec/select_in_tmpl.go +++ b/pkg/sql/colexec/select_in_tmpl.go @@ -51,7 +51,7 @@ const _CANONICAL_TYPE_FAMILY = types.UnknownFamily const _TYPE_WIDTH = 0 func _COMPARE(_, _, _, _, _ string) bool { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/selection_ops_tmpl.go b/pkg/sql/colexec/selection_ops_tmpl.go index 631de30faa98..dd655c10fc5c 100644 --- a/pkg/sql/colexec/selection_ops_tmpl.go +++ b/pkg/sql/colexec/selection_ops_tmpl.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // {{/* @@ -49,7 +50,7 @@ const _RIGHT_TYPE_WIDTH = 0 // _ASSIGN_CMP is the template function for assigning the result of comparing // the second input to the third input into the first input. func _ASSIGN_CMP(_, _, _, _, _, _ interface{}) int { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/sort.eg.go b/pkg/sql/colexec/sort.eg.go index c31a365119b3..fda51770b526 100644 --- a/pkg/sql/colexec/sort.eg.go +++ b/pkg/sql/colexec/sort.eg.go @@ -12,7 +12,6 @@ package colexec import ( "bytes" "context" - "fmt" "math" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -22,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) func isSorterSupported(t *types.T, dir execinfrapb.Ordering_Column_Direction) bool { @@ -371,7 +371,7 @@ func newSingleSorter( } } } - colexecerror.InternalError("isSorterSupported should have caught this") + colexecerror.InternalError(errors.AssertionFailedf("isSorterSupported should have caught this")) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -396,7 +396,7 @@ func (s *sortBoolAscWithNullsOp) sort(ctx context.Context) { func (s *sortBoolAscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -474,7 +474,7 @@ func (s *sortBytesAscWithNullsOp) sort(ctx context.Context) { func (s *sortBytesAscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -544,7 +544,7 @@ func (s *sortDecimalAscWithNullsOp) sort(ctx context.Context) { func (s *sortDecimalAscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -614,7 +614,7 @@ func (s *sortInt16AscWithNullsOp) sort(ctx context.Context) { func (s *sortInt16AscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -695,7 +695,7 @@ func (s *sortInt32AscWithNullsOp) sort(ctx context.Context) { func (s *sortInt32AscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -776,7 +776,7 @@ func (s *sortInt64AscWithNullsOp) sort(ctx context.Context) { func (s *sortInt64AscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -857,7 +857,7 @@ func (s *sortFloat64AscWithNullsOp) sort(ctx context.Context) { func (s *sortFloat64AscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -946,7 +946,7 @@ func (s *sortTimestampAscWithNullsOp) sort(ctx context.Context) { func (s *sortTimestampAscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1023,7 +1023,7 @@ func (s *sortIntervalAscWithNullsOp) sort(ctx context.Context) { func (s *sortIntervalAscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1093,7 +1093,7 @@ func (s *sortDatumAscWithNullsOp) sort(ctx context.Context) { func (s *sortDatumAscWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1165,7 +1165,7 @@ func (s *sortBoolDescWithNullsOp) sort(ctx context.Context) { func (s *sortBoolDescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1243,7 +1243,7 @@ func (s *sortBytesDescWithNullsOp) sort(ctx context.Context) { func (s *sortBytesDescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1313,7 +1313,7 @@ func (s *sortDecimalDescWithNullsOp) sort(ctx context.Context) { func (s *sortDecimalDescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1383,7 +1383,7 @@ func (s *sortInt16DescWithNullsOp) sort(ctx context.Context) { func (s *sortInt16DescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1464,7 +1464,7 @@ func (s *sortInt32DescWithNullsOp) sort(ctx context.Context) { func (s *sortInt32DescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1545,7 +1545,7 @@ func (s *sortInt64DescWithNullsOp) sort(ctx context.Context) { func (s *sortInt64DescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1626,7 +1626,7 @@ func (s *sortFloat64DescWithNullsOp) sort(ctx context.Context) { func (s *sortFloat64DescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1715,7 +1715,7 @@ func (s *sortTimestampDescWithNullsOp) sort(ctx context.Context) { func (s *sortTimestampDescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1792,7 +1792,7 @@ func (s *sortIntervalDescWithNullsOp) sort(ctx context.Context) { func (s *sortIntervalDescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1862,7 +1862,7 @@ func (s *sortDatumDescWithNullsOp) sort(ctx context.Context) { func (s *sortDatumDescWithNullsOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -1934,7 +1934,7 @@ func (s *sortBoolAscOp) sort(ctx context.Context) { func (s *sortBoolAscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2002,7 +2002,7 @@ func (s *sortBytesAscOp) sort(ctx context.Context) { func (s *sortBytesAscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2062,7 +2062,7 @@ func (s *sortDecimalAscOp) sort(ctx context.Context) { func (s *sortDecimalAscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2122,7 +2122,7 @@ func (s *sortInt16AscOp) sort(ctx context.Context) { func (s *sortInt16AscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2193,7 +2193,7 @@ func (s *sortInt32AscOp) sort(ctx context.Context) { func (s *sortInt32AscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2264,7 +2264,7 @@ func (s *sortInt64AscOp) sort(ctx context.Context) { func (s *sortInt64AscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2335,7 +2335,7 @@ func (s *sortFloat64AscOp) sort(ctx context.Context) { func (s *sortFloat64AscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2414,7 +2414,7 @@ func (s *sortTimestampAscOp) sort(ctx context.Context) { func (s *sortTimestampAscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2481,7 +2481,7 @@ func (s *sortIntervalAscOp) sort(ctx context.Context) { func (s *sortIntervalAscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2541,7 +2541,7 @@ func (s *sortDatumAscOp) sort(ctx context.Context) { func (s *sortDatumAscOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2603,7 +2603,7 @@ func (s *sortBoolDescOp) sort(ctx context.Context) { func (s *sortBoolDescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2671,7 +2671,7 @@ func (s *sortBytesDescOp) sort(ctx context.Context) { func (s *sortBytesDescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2731,7 +2731,7 @@ func (s *sortDecimalDescOp) sort(ctx context.Context) { func (s *sortDecimalDescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2791,7 +2791,7 @@ func (s *sortInt16DescOp) sort(ctx context.Context) { func (s *sortInt16DescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2862,7 +2862,7 @@ func (s *sortInt32DescOp) sort(ctx context.Context) { func (s *sortInt32DescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -2933,7 +2933,7 @@ func (s *sortInt64DescOp) sort(ctx context.Context) { func (s *sortInt64DescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -3004,7 +3004,7 @@ func (s *sortFloat64DescOp) sort(ctx context.Context) { func (s *sortFloat64DescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -3083,7 +3083,7 @@ func (s *sortTimestampDescOp) sort(ctx context.Context) { func (s *sortTimestampDescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -3150,7 +3150,7 @@ func (s *sortIntervalDescOp) sort(ctx context.Context) { func (s *sortIntervalDescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { @@ -3210,7 +3210,7 @@ func (s *sortDatumDescOp) sort(ctx context.Context) { func (s *sortDatumDescOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { diff --git a/pkg/sql/colexec/sort.go b/pkg/sql/colexec/sort.go index 78e55f84eb9f..7ffbe5f26891 100644 --- a/pkg/sql/colexec/sort.go +++ b/pkg/sql/colexec/sort.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -133,7 +132,7 @@ func (p *allSpooler) init() { func (p *allSpooler) spool(ctx context.Context) { if p.spooled { - colexecerror.InternalError("spool() is called for the second time") + colexecerror.InternalError(errors.AssertionFailedf("spool() is called for the second time")) } p.spooled = true for batch := p.input.Next(ctx); batch.Length() != 0; batch = p.input.Next(ctx) { @@ -145,7 +144,7 @@ func (p *allSpooler) spool(ctx context.Context) { func (p *allSpooler) getValues(i int) coldata.Vec { if !p.spooled { - colexecerror.InternalError("getValues() is called before spool()") + colexecerror.InternalError(errors.AssertionFailedf("getValues() is called before spool()")) } return p.bufferedTuples.ColVec(i) } @@ -156,7 +155,7 @@ func (p *allSpooler) getNumTuples() int { func (p *allSpooler) getPartitionsCol() []bool { if !p.spooled { - colexecerror.InternalError("getPartitionsCol() is called before spool()") + colexecerror.InternalError(errors.AssertionFailedf("getPartitionsCol() is called before spool()")) } return nil } @@ -297,7 +296,7 @@ func (p *sortOp) Next(ctx context.Context) coldata.Batch { case sortDone: return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("invalid sort state %v", p.state)) + colexecerror.InternalError(errors.AssertionFailedf("invalid sort state %v", p.state)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -418,7 +417,7 @@ func (p *sortOp) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return p.input } - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/sort_chunks.go b/pkg/sql/colexec/sort_chunks.go index 1b52de6b9953..6aa06c613b42 100644 --- a/pkg/sql/colexec/sort_chunks.go +++ b/pkg/sql/colexec/sort_chunks.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -21,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // NewSortChunks returns a new sort chunks operator, which sorts its input on @@ -35,7 +35,7 @@ func NewSortChunks( matchLen int, ) (colexecbase.Operator, error) { if matchLen < 1 || matchLen == len(orderingCols) { - colexecerror.InternalError(fmt.Sprintf( + colexecerror.InternalError(errors.AssertionFailedf( "sort chunks should only be used when the input is "+ "already ordered on at least one column but not fully ordered; "+ "num ordering cols = %d, matchLen = %d", len(orderingCols), matchLen)) @@ -76,7 +76,7 @@ func (c *sortChunksOp) Child(nth int, verbose bool) execinfra.OpNode { if nth == 0 { return c.input } - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -311,7 +311,7 @@ func (s *chunker) prepareNextChunks(ctx context.Context) chunkerReadingState { if s.batch.Selection() != nil { // We assume that the input has been deselected, so the batch should // never have a selection vector set. - colexecerror.InternalError(fmt.Sprintf("unexpected: batch with non-nil selection vector")) + colexecerror.InternalError(errors.AssertionFailedf("unexpected: batch with non-nil selection vector")) } // First, run the partitioners on our pre-sorted columns to determine the @@ -404,11 +404,11 @@ func (s *chunker) prepareNextChunks(ctx context.Context) chunkerReadingState { if s.inputDone { return chunkerDone } - colexecerror.InternalError(fmt.Sprintf("unexpected: chunkerEmittingFromBatch state" + + colexecerror.InternalError(errors.AssertionFailedf("unexpected: chunkerEmittingFromBatch state" + "when s.chunks is fully processed and input is not done")) } default: - colexecerror.InternalError(fmt.Sprintf("invalid chunker spooler state %v", s.state)) + colexecerror.InternalError(errors.AssertionFailedf("invalid chunker spooler state %v", s.state)) } } } @@ -436,7 +436,7 @@ func (s *chunker) getValues(i int) coldata.Vec { case chunkerReadFromBatch: return s.batch.ColVec(i).Window(s.chunks[s.chunksStartIdx], s.chunks[len(s.chunks)-1]) default: - colexecerror.InternalError(fmt.Sprintf("unexpected chunkerReadingState in getValues: %v", s.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected chunkerReadingState in getValues: %v", s.state)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -451,7 +451,7 @@ func (s *chunker) getNumTuples() int { case chunkerDone: return 0 default: - colexecerror.InternalError(fmt.Sprintf("unexpected chunkerReadingState in getNumTuples: %v", s.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected chunkerReadingState in getNumTuples: %v", s.state)) // This code is unreachable, but the compiler cannot infer that. return 0 } @@ -479,14 +479,14 @@ func (s *chunker) getPartitionsCol() []bool { case chunkerDone: return nil default: - colexecerror.InternalError(fmt.Sprintf("unexpected chunkerReadingState in getPartitionsCol: %v", s.state)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected chunkerReadingState in getPartitionsCol: %v", s.state)) // This code is unreachable, but the compiler cannot infer that. return nil } } func (s *chunker) getWindowedBatch(startIdx, endIdx int) coldata.Batch { - colexecerror.InternalError("getWindowedBatch is not implemented on chunker spooler") + colexecerror.InternalError(errors.AssertionFailedf("getWindowedBatch is not implemented on chunker spooler")) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/sort_test.go b/pkg/sql/colexec/sort_test.go index af5f3444ca9c..76a108c1dfd0 100644 --- a/pkg/sql/colexec/sort_test.go +++ b/pkg/sql/colexec/sort_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/cockroachdb/errors" ) var sortAllTestCases []sortTestCase @@ -402,7 +403,7 @@ func generateColumnOrdering( rng *rand.Rand, nCols int, nOrderingCols int, ) []execinfrapb.Ordering_Column { if nOrderingCols > nCols { - colexecerror.InternalError("nOrderingCols > nCols in generateColumnOrdering") + colexecerror.InternalError(errors.AssertionFailedf("nOrderingCols > nCols in generateColumnOrdering")) } orderingCols := make([]execinfrapb.Ordering_Column, nOrderingCols) for i, col := range rng.Perm(nCols)[:nOrderingCols] { diff --git a/pkg/sql/colexec/sort_tmpl.go b/pkg/sql/colexec/sort_tmpl.go index eca9d48f47ab..a65639535716 100644 --- a/pkg/sql/colexec/sort_tmpl.go +++ b/pkg/sql/colexec/sort_tmpl.go @@ -21,13 +21,13 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // {{/* @@ -53,7 +53,7 @@ const _ISNULL = false // _ASSIGN_LT is the template equality function for assigning the first input // to the result of the second input < the third input. func _ASSIGN_LT(_, _, _, _, _, _ string) bool { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} @@ -108,7 +108,7 @@ func newSingleSorter( } // {{end}} } - colexecerror.InternalError("isSorterSupported should have caught this") + colexecerror.InternalError(errors.AssertionFailedf("isSorterSupported should have caught this")) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -140,7 +140,7 @@ func (s *sort_TYPE_DIR_HANDLES_NULLSOp) sort(ctx context.Context) { func (s *sort_TYPE_DIR_HANDLES_NULLSOp) sortPartitions(ctx context.Context, partitions []int) { if len(partitions) < 1 { - colexecerror.InternalError(fmt.Sprintf("invalid partitions list %v", partitions)) + colexecerror.InternalError(errors.AssertionFailedf("invalid partitions list %v", partitions)) } order := s.order for i, partitionStart := range partitions { diff --git a/pkg/sql/colexec/sorttopk.go b/pkg/sql/colexec/sorttopk.go index 7bb56afc7beb..058470f3308f 100644 --- a/pkg/sql/colexec/sorttopk.go +++ b/pkg/sql/colexec/sorttopk.go @@ -13,7 +13,6 @@ package colexec import ( "container/heap" "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -21,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) const ( @@ -125,7 +125,7 @@ func (t *topKSorter) Next(ctx context.Context) coldata.Batch { case topKSortDone: return coldata.ZeroBatch default: - colexecerror.InternalError(fmt.Sprintf("invalid sort state %v", t.state)) + colexecerror.InternalError(errors.AssertionFailedf("invalid sort state %v", t.state)) // This code is unreachable, but the compiler cannot infer that. return nil } @@ -251,7 +251,7 @@ func (t *topKSorter) compareRow(vecIdx1, vecIdx2 int, rowIdx1, rowIdx2 int) int case execinfrapb.Ordering_Column_DESC: return -res default: - colexecerror.InternalError(fmt.Sprintf("unexpected direction value %d", d)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected direction value %d", d)) } } } diff --git a/pkg/sql/colexec/spilling_queue.go b/pkg/sql/colexec/spilling_queue.go index bdc03d8334cd..525b7f11eb5e 100644 --- a/pkg/sql/colexec/spilling_queue.go +++ b/pkg/sql/colexec/spilling_queue.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colcontainer" @@ -166,7 +165,7 @@ func (q *spillingQueue) dequeue(ctx context.Context) (coldata.Batch, error) { // No more in-memory items. Fill the circular buffer as much as possible. // Note that there must be at least one element on disk. if !q.rewindable && q.curHeadIdx != q.curTailIdx { - colexecerror.InternalError(fmt.Sprintf("assertion failed in spillingQueue: curHeadIdx != curTailIdx, %d != %d", q.curHeadIdx, q.curTailIdx)) + colexecerror.InternalError(errors.AssertionFailedf("assertion failed in spillingQueue: curHeadIdx != curTailIdx, %d != %d", q.curHeadIdx, q.curTailIdx)) } // NOTE: Only one item is dequeued from disk since a deserialized batch is // only valid until the next call to Dequeue. In practice we could Dequeue @@ -182,7 +181,7 @@ func (q *spillingQueue) dequeue(ctx context.Context) (coldata.Batch, error) { if !ok { // There was no batch to dequeue from disk. This should not really // happen, as it should have been caught by the q.empty() check above. - colexecerror.InternalError("disk queue was not empty but failed to dequeue element in spillingQueue") + colexecerror.InternalError(errors.AssertionFailedf("disk queue was not empty but failed to dequeue element in spillingQueue")) } // Account for this batch's memory. q.unlimitedAllocator.RetainBatch(q.dequeueScratch) diff --git a/pkg/sql/colexec/stats.go b/pkg/sql/colexec/stats.go index 1eddd38bd5c3..d6d768c97240 100644 --- a/pkg/sql/colexec/stats.go +++ b/pkg/sql/colexec/stats.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/errors" ) // VectorizedStatsCollector collects VectorizedStats on Operators. @@ -70,7 +71,7 @@ func NewVectorizedStatsCollector( inputStatsCollectors []*VectorizedStatsCollector, ) *VectorizedStatsCollector { if inputWatch == nil { - colexecerror.InternalError("input watch for VectorizedStatsCollector is nil") + colexecerror.InternalError(errors.AssertionFailedf("input watch for VectorizedStatsCollector is nil")) } // ioTime indicates whether the time should be displayed as "IO time" on // the diagram. diff --git a/pkg/sql/colexec/substring.eg.go b/pkg/sql/colexec/substring.eg.go index 067a9a1bd3d3..cd4667b94acb 100644 --- a/pkg/sql/colexec/substring.eg.go +++ b/pkg/sql/colexec/substring.eg.go @@ -11,7 +11,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -37,10 +36,10 @@ func newSubstringOperator( outputIdx: outputIdx, } if startType.Family() != types.IntFamily { - colexecerror.InternalError(fmt.Sprintf("non-int start argument type %s", startType)) + colexecerror.InternalError(errors.AssertionFailedf("non-int start argument type %s", startType)) } if lengthType.Family() != types.IntFamily { - colexecerror.InternalError(fmt.Sprintf("non-int length argument type %s", lengthType)) + colexecerror.InternalError(errors.AssertionFailedf("non-int length argument type %s", lengthType)) } switch startType.Width() { case -1: diff --git a/pkg/sql/colexec/substring_tmpl.go b/pkg/sql/colexec/substring_tmpl.go index 6aa25f3bd272..b5891029c44e 100644 --- a/pkg/sql/colexec/substring_tmpl.go +++ b/pkg/sql/colexec/substring_tmpl.go @@ -21,7 +21,6 @@ package colexec import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -57,10 +56,10 @@ func newSubstringOperator( outputIdx: outputIdx, } if startType.Family() != types.IntFamily { - colexecerror.InternalError(fmt.Sprintf("non-int start argument type %s", startType)) + colexecerror.InternalError(errors.AssertionFailedf("non-int start argument type %s", startType)) } if lengthType.Family() != types.IntFamily { - colexecerror.InternalError(fmt.Sprintf("non-int length argument type %s", lengthType)) + colexecerror.InternalError(errors.AssertionFailedf("non-int length argument type %s", lengthType)) } switch startType.Width() { // {{range $startWidth, $lengthWidths := .}} diff --git a/pkg/sql/colexec/sum_agg_tmpl.go b/pkg/sql/colexec/sum_agg_tmpl.go index f789668a2d6b..fc36df6eafe3 100644 --- a/pkg/sql/colexec/sum_agg_tmpl.go +++ b/pkg/sql/colexec/sum_agg_tmpl.go @@ -36,7 +36,7 @@ import ( // _ASSIGN_ADD is the template addition function for assigning the first input // to the result of the second input + the third input. func _ASSIGN_ADD(_, _, _, _, _, _ string) { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} diff --git a/pkg/sql/colexec/utils.eg.go b/pkg/sql/colexec/utils.eg.go index b670fe66528c..5143298c1da7 100644 --- a/pkg/sql/colexec/utils.eg.go +++ b/pkg/sql/colexec/utils.eg.go @@ -10,13 +10,12 @@ package colexec import ( - "fmt" - "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/errors" ) // GetDatumToPhysicalFn returns a function for converting a datum of the given @@ -160,7 +159,7 @@ func GetDatumToPhysicalFn(ct *types.T) func(tree.Datum) interface{} { } } } - colexecerror.InternalError(fmt.Sprintf("unexpectedly unhandled type %s", ct.DebugString())) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly unhandled type %s", ct.DebugString())) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/utils.go b/pkg/sql/colexec/utils.go index 9f0fca05fd22..036e7fdd1aa0 100644 --- a/pkg/sql/colexec/utils.go +++ b/pkg/sql/colexec/utils.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/errors" ) var ( @@ -183,11 +184,11 @@ func (b *appendOnlyBufferedBatch) ColVecs() []coldata.Vec { } func (b *appendOnlyBufferedBatch) AppendCol(coldata.Vec) { - colexecerror.InternalError("AppendCol is prohibited on appendOnlyBufferedBatch") + colexecerror.InternalError(errors.AssertionFailedf("AppendCol is prohibited on appendOnlyBufferedBatch")) } func (b *appendOnlyBufferedBatch) ReplaceCol(coldata.Vec, int) { - colexecerror.InternalError("ReplaceCol is prohibited on appendOnlyBufferedBatch") + colexecerror.InternalError(errors.AssertionFailedf("ReplaceCol is prohibited on appendOnlyBufferedBatch")) } // append is a helper method that appends all tuples with indices in range diff --git a/pkg/sql/colexec/utils_test.go b/pkg/sql/colexec/utils_test.go index ff706ece4f9d..f3ee1ad03356 100644 --- a/pkg/sql/colexec/utils_test.go +++ b/pkg/sql/colexec/utils_test.go @@ -139,7 +139,7 @@ func (t tuple) less(other tuple, evalCtx *tree.EvalContext) bool { case "string": return lhsVal.String() < rhsVal.String() default: - colexecerror.InternalError(fmt.Sprintf("Unhandled comparison type: %s", typ)) + colexecerror.InternalError(errors.AssertionFailedf("Unhandled comparison type: %s", typ)) } } return false @@ -584,7 +584,7 @@ func setColVal(vec coldata.Vec, idx int, val interface{}, evalCtx *tree.EvalCont decimalVal, _, err := apd.NewFromString(fmt.Sprintf("%f", floatVal)) if err != nil { colexecerror.InternalError( - fmt.Sprintf("unable to set decimal %f: %v", floatVal, err)) + errors.AssertionFailedf("unable to set decimal %f: %v", floatVal, err)) } // .Set is used here instead of assignment to ensure the pointer address // of the underlying storage for apd.Decimal remains the same. This can @@ -602,7 +602,7 @@ func setColVal(vec coldata.Vec, idx int, val interface{}, evalCtx *tree.EvalCont case json.JSON: vec.Datum().Set(idx, &tree.DJSON{JSON: v}) default: - colexecerror.InternalError(fmt.Sprintf("unexpected type %T of datum-backed value: %v", v, v)) + colexecerror.InternalError(errors.AssertionFailedf("unexpected type %T of datum-backed value: %v", v, v)) } } else { reflect.ValueOf(vec.Col()).Index(idx).Set(reflect.ValueOf(val).Convert(reflect.TypeOf(vec.Col()).Elem())) @@ -693,7 +693,7 @@ func newOpTestSelInput(rng *rand.Rand, batchSize int, tuples tuples, typs []*typ func (s *opTestInput) Init() { if s.typs == nil { if len(s.tuples) == 0 { - colexecerror.InternalError("empty tuple source with no specified types") + colexecerror.InternalError(errors.AssertionFailedf("empty tuple source with no specified types")) } s.typs = extrapolateTypesFromTuples(s.tuples) } @@ -720,7 +720,7 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { tupleLen := len(tups[0]) for i := range tups { if len(tups[i]) != tupleLen { - colexecerror.InternalError(fmt.Sprintf("mismatched tuple lens: found %+v expected %d vals", + colexecerror.InternalError(errors.AssertionFailedf("mismatched tuple lens: found %+v expected %d vals", tups[i], tupleLen)) } } @@ -792,7 +792,7 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { d := apd.Decimal{} _, err := d.SetFloat64(rng.Float64()) if err != nil { - colexecerror.InternalError(fmt.Sprintf("%v", err)) + colexecerror.InternalError(errors.AssertionFailedf("%v", err)) } col.Index(outputIdx).Set(reflect.ValueOf(d)) case types.BytesFamily: @@ -811,13 +811,13 @@ func (s *opTestInput) Next(context.Context) coldata.Batch { case types.TupleFamily: setColVal(vec, outputIdx, stringToDatum("(NULL)", vec.Type(), s.evalCtx), s.evalCtx) default: - colexecerror.InternalError(fmt.Sprintf("unexpected datum-backed type: %s", vec.Type())) + colexecerror.InternalError(errors.AssertionFailedf("unexpected datum-backed type: %s", vec.Type())) } default: if val, ok := quick.Value(reflect.TypeOf(vec.Col()).Elem(), rng); ok { setColVal(vec, outputIdx, val.Interface(), s.evalCtx) } else { - colexecerror.InternalError(fmt.Sprintf("could not generate a random value of type %s", vec.Type())) + colexecerror.InternalError(errors.AssertionFailedf("could not generate a random value of type %s", vec.Type())) } } } @@ -868,7 +868,7 @@ func newOpFixedSelTestInput( func (s *opFixedSelTestInput) Init() { if s.typs == nil { if len(s.tuples) == 0 { - colexecerror.InternalError("empty tuple source with no specified types") + colexecerror.InternalError(errors.AssertionFailedf("empty tuple source with no specified types")) } s.typs = extrapolateTypesFromTuples(s.tuples) } @@ -877,7 +877,7 @@ func (s *opFixedSelTestInput) Init() { tupleLen := len(s.tuples[0]) for _, i := range s.sel { if len(s.tuples[i]) != tupleLen { - colexecerror.InternalError(fmt.Sprintf("mismatched tuple lens: found %+v expected %d vals", + colexecerror.InternalError(errors.AssertionFailedf("mismatched tuple lens: found %+v expected %d vals", s.tuples[i], tupleLen)) } } diff --git a/pkg/sql/colexec/utils_tmpl.go b/pkg/sql/colexec/utils_tmpl.go index 6e78a87445c1..de11fd088e9d 100644 --- a/pkg/sql/colexec/utils_tmpl.go +++ b/pkg/sql/colexec/utils_tmpl.go @@ -20,11 +20,10 @@ package colexec import ( - "fmt" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // GetDatumToPhysicalFn returns a function for converting a datum of the given @@ -48,7 +47,7 @@ func GetDatumToPhysicalFn(ct *types.T) func(tree.Datum) interface{} { } // {{end}} } - colexecerror.InternalError(fmt.Sprintf("unexpectedly unhandled type %s", ct.DebugString())) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly unhandled type %s", ct.DebugString())) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/values_differ.eg.go b/pkg/sql/colexec/values_differ.eg.go index 4b27d9077902..b8abe99ab361 100644 --- a/pkg/sql/colexec/values_differ.eg.go +++ b/pkg/sql/colexec/values_differ.eg.go @@ -11,7 +11,6 @@ package colexec import ( "bytes" - "fmt" "math" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -20,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // valuesDiffer takes in two ColVecs as well as values indices to check whether @@ -359,7 +359,7 @@ func valuesDiffer(aColVec coldata.Vec, aValueIdx int, bColVec coldata.Vec, bValu return unique } } - colexecerror.InternalError(fmt.Sprintf("unsupported valuesDiffer type %s", aColVec.Type())) + colexecerror.InternalError(errors.AssertionFailedf("unsupported valuesDiffer type %s", aColVec.Type())) // This code is unreachable, but the compiler cannot infer that. return false } diff --git a/pkg/sql/colexec/values_differ_tmpl.go b/pkg/sql/colexec/values_differ_tmpl.go index 9f4d94a5f620..9cfd0bc8479f 100644 --- a/pkg/sql/colexec/values_differ_tmpl.go +++ b/pkg/sql/colexec/values_differ_tmpl.go @@ -20,11 +20,10 @@ package colexec import ( - "fmt" - "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // {{/* @@ -43,7 +42,7 @@ const _TYPE_WIDTH = 0 // _ASSIGN_NE is the template equality function for assigning the first input // to the result of the second input != the third input. func _ASSIGN_NE(_, _, _, _, _, _ string) bool { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} @@ -78,7 +77,7 @@ func valuesDiffer(aColVec coldata.Vec, aValueIdx int, bColVec coldata.Vec, bValu } // {{end}} } - colexecerror.InternalError(fmt.Sprintf("unsupported valuesDiffer type %s", aColVec.Type())) + colexecerror.InternalError(errors.AssertionFailedf("unsupported valuesDiffer type %s", aColVec.Type())) // This code is unreachable, but the compiler cannot infer that. return false } diff --git a/pkg/sql/colexec/vec_comparators.eg.go b/pkg/sql/colexec/vec_comparators.eg.go index 6942769f29df..9c656c942230 100644 --- a/pkg/sql/colexec/vec_comparators.eg.go +++ b/pkg/sql/colexec/vec_comparators.eg.go @@ -11,7 +11,6 @@ package colexec import ( "bytes" - "fmt" "math" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -20,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // vecComparator is a helper for the ordered synchronizer. It stores multiple @@ -570,7 +570,7 @@ func GetVecComparator(t *types.T, numVecs int) vecComparator { } } } - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", t)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", t)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexec/vec_comparators_tmpl.go b/pkg/sql/colexec/vec_comparators_tmpl.go index c00158407565..13643b8ffbef 100644 --- a/pkg/sql/colexec/vec_comparators_tmpl.go +++ b/pkg/sql/colexec/vec_comparators_tmpl.go @@ -20,13 +20,12 @@ package colexec import ( - "fmt" - "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" ) // {{/* @@ -45,7 +44,7 @@ const _TYPE_WIDTH = 0 // _COMPARE is the template equality function for assigning the first input // to the result of comparing second and third inputs. func _COMPARE(_, _, _, _, _ string) bool { - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } // */}} @@ -136,7 +135,7 @@ func GetVecComparator(t *types.T, numVecs int) vecComparator { } // {{end}} } - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", t)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", t)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexecbase/colexecerror/error.go b/pkg/sql/colexecbase/colexecerror/error.go index 13602b61a0ec..d37d2fcdcf5b 100644 --- a/pkg/sql/colexecbase/colexecerror/error.go +++ b/pkg/sql/colexecbase/colexecerror/error.go @@ -174,7 +174,7 @@ func init() { // caught and returned as internal error to the client with the corresponding // stack trace. This method should be called to propagate errors that resulted // in the vectorized engine being in an *unexpected* state. -func InternalError(err interface{}) { +func InternalError(err error) { panic(err) } diff --git a/pkg/sql/colexecbase/operator.go b/pkg/sql/colexecbase/operator.go index 545f25f5c1c4..67fbf098f44c 100644 --- a/pkg/sql/colexecbase/operator.go +++ b/pkg/sql/colexecbase/operator.go @@ -12,12 +12,12 @@ package colexecbase import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/errors" ) // Operator is a column vector operator that produces a Batch as output. @@ -64,7 +64,7 @@ func (ZeroInputNode) ChildCount(verbose bool) int { // Child implements the execinfra.OpNode interface. func (ZeroInputNode) Child(nth int, verbose bool) execinfra.OpNode { - colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) + colexecerror.InternalError(errors.AssertionFailedf("invalid index %d", nth)) // This code is unreachable, but the compiler cannot infer that. return nil } diff --git a/pkg/sql/colexecbase/testutils.go b/pkg/sql/colexecbase/testutils.go index bd55cddc2f30..54c7e3057419 100644 --- a/pkg/sql/colexecbase/testutils.go +++ b/pkg/sql/colexecbase/testutils.go @@ -12,7 +12,6 @@ package colexecbase import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" @@ -192,10 +191,10 @@ func (s *TestingSemaphore) TryAcquire(n int) bool { // Release implements the semaphore.Semaphore interface. func (s *TestingSemaphore) Release(n int) int { if n < 0 { - colexecerror.InternalError("releasing a negative amount") + colexecerror.InternalError(errors.AssertionFailedf("releasing a negative amount")) } if s.count-n < 0 { - colexecerror.InternalError(fmt.Sprintf("testing semaphore too many resources released, releasing %d, have %d", n, s.count)) + colexecerror.InternalError(errors.AssertionFailedf("testing semaphore too many resources released, releasing %d, have %d", n, s.count)) } pre := s.count s.count -= n diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index 4eba3224bd31..45075414f593 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -81,7 +81,7 @@ func (s *ColBatchScan) Next(ctx context.Context) coldata.Batch { colexecerror.InternalError(err) } if bat.Selection() != nil { - colexecerror.InternalError("unexpectedly a selection vector is set on the batch coming from CFetcher") + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly a selection vector is set on the batch coming from CFetcher")) } s.rowsRead += bat.Length() return bat diff --git a/pkg/sql/colflow/colrpc/inbox.go b/pkg/sql/colflow/colrpc/inbox.go index f8161502c71e..90b08e8d3560 100644 --- a/pkg/sql/colflow/colrpc/inbox.go +++ b/pkg/sql/colflow/colrpc/inbox.go @@ -245,7 +245,7 @@ func (i *Inbox) Next(ctx context.Context) coldata.Batch { // during normal termination. if err := recover(); err != nil { i.close() - colexecerror.InternalError(err) + colexecerror.InternalError(log.PanicAsError(0, err)) } }() diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index 6059049fc7c3..46205db1f40c 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -305,7 +305,7 @@ func (f *vectorizedFlow) Cleanup(ctx context.Context) { if f.Cfg.TestingKnobs.CheckVectorizedFlowIsClosedCorrectly { if numClosed := atomic.LoadInt32(f.testingInfo.numClosed); numClosed != f.testingInfo.numClosers { - colexecerror.InternalError(fmt.Sprintf("expected %d components to be closed, but found that only %d were", f.testingInfo.numClosers, numClosed)) + colexecerror.InternalError(errors.AssertionFailedf("expected %d components to be closed, but found that only %d were", f.testingInfo.numClosers, numClosed)) } } @@ -1123,7 +1123,7 @@ func (s *vectorizedFlowCreator) setupFlow( } if len(s.vectorizedStatsCollectorsQueue) > 0 { - colexecerror.InternalError("not all vectorized stats collectors have been processed") + colexecerror.InternalError(errors.AssertionFailedf("not all vectorized stats collectors have been processed")) } return s.leaves, nil } diff --git a/pkg/sql/colflow/vectorized_panic_propagation_test.go b/pkg/sql/colflow/vectorized_panic_propagation_test.go index 4fb057184fbd..03ee35899656 100644 --- a/pkg/sql/colflow/vectorized_panic_propagation_test.go +++ b/pkg/sql/colflow/vectorized_panic_propagation_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -145,7 +146,7 @@ func (e *testVectorizedInternalPanicEmitter) Init() { func (e *testVectorizedInternalPanicEmitter) Next(ctx context.Context) coldata.Batch { if !e.emitBatch { e.emitBatch = true - colexecerror.InternalError("") + colexecerror.InternalError(errors.AssertionFailedf("")) } e.emitBatch = false diff --git a/pkg/sql/colmem/allocator.go b/pkg/sql/colmem/allocator.go index cac5b12fabf0..98c1f84aa60c 100644 --- a/pkg/sql/colmem/allocator.go +++ b/pkg/sql/colmem/allocator.go @@ -12,7 +12,6 @@ package colmem import ( "context" - "fmt" "time" "unsafe" @@ -228,7 +227,7 @@ func (a *Allocator) NewMemColumn(t *types.T, capacity int) coldata.Vec { // NOTE: b must be non-zero length batch. func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { if b.Length() == 0 { - colexecerror.InternalError("trying to add a column to zero length batch") + colexecerror.InternalError(errors.AssertionFailedf("trying to add a column to zero length batch")) } width := b.Width() if colIdx < width { @@ -250,14 +249,14 @@ func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { return } // We have a vector with an unexpected type, so we panic. - colexecerror.InternalError(errors.Errorf( + colexecerror.InternalError(errors.AssertionFailedf( "trying to add a column of %s type at index %d but %s vector already present", t, colIdx, presentType, )) } else if colIdx > width { // We have a batch of unexpected width which indicates an error in the // planning stage. - colexecerror.InternalError(errors.Errorf( + colexecerror.InternalError(errors.AssertionFailedf( "trying to add a column of %s type at index %d but batch has width %d", t, colIdx, width, )) @@ -305,7 +304,7 @@ func (a *Allocator) AdjustMemoryUsage(delta int64) { // allocator by (at most) size bytes. size must be non-negative. func (a *Allocator) ReleaseMemory(size int64) { if size < 0 { - colexecerror.InternalError(fmt.Sprintf("unexpectedly negative size in ReleaseMemory: %d", size)) + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly negative size in ReleaseMemory: %d", size)) } if size > a.acc.Used() { size = a.acc.Used() @@ -382,7 +381,7 @@ func EstimateBatchSizeBytes(vecTypes []*types.T, batchLength int) int { implementationSize, _ := tree.DatumTypeSize(t) acc += int(implementationSize) + sizeOfDatum default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", t)) + colexecerror.InternalError(errors.AssertionFailedf("unhandled type %s", t)) } } // For byte arrays, we initially allocate BytesInitialAllocationFactor diff --git a/pkg/sql/opaque.go b/pkg/sql/opaque.go index d721d69e48bd..35123c6640b0 100644 --- a/pkg/sql/opaque.go +++ b/pkg/sql/opaque.go @@ -12,7 +12,6 @@ package sql import ( "context" - "fmt" "reflect" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -164,10 +163,7 @@ func buildOpaque( plan, err = p.maybePlanHook(ctx, stmt) if plan == nil && err == nil { return nil, pgerror.Newf(pgcode.CCLRequired, - "a CCL binary is required to use this statement type: %s", - // TODO(knz): Simplify this to %T. - // See: https://github.com/cockroachdb/cockroach/issues/53207 - errors.Safe(fmt.Sprintf("%T", stmt))) + "a CCL binary is required to use this statement type: %T", stmt) } default: return nil, errors.AssertionFailedf("unknown opaque statement %T", stmt) diff --git a/pkg/sql/sem/tree/as_of.go b/pkg/sql/sem/tree/as_of.go index f0417cb0b771..4dfc917a6197 100644 --- a/pkg/sql/sem/tree/as_of.go +++ b/pkg/sql/sem/tree/as_of.go @@ -12,7 +12,6 @@ package tree import ( "context" - "fmt" "strconv" "strings" "time" @@ -127,9 +126,7 @@ func DatumToHLC(evalCtx *EvalContext, stmtTimestamp time.Time, d Datum) (hlc.Tim default: convErr = errors.WithSafeDetails( errors.Errorf("expected timestamp, decimal, or interval, got %s", d.ResolvedType()), - // TODO(knz): Simplify this to %T. - // See https://github.com/cockroachdb/cockroach/issues/53207 - "go type: %s", errors.Safe(fmt.Sprintf("%T", d))) + "go type: %T", d) } if convErr != nil { return ts, convErr diff --git a/pkg/storage/enginepb/mvcc3.go b/pkg/storage/enginepb/mvcc3.go index b6d2c9d2d5ba..d64ae1ddcdf9 100644 --- a/pkg/storage/enginepb/mvcc3.go +++ b/pkg/storage/enginepb/mvcc3.go @@ -10,7 +10,7 @@ package enginepb -import "fmt" +import "github.com/cockroachdb/errors" // ToStats converts the receiver to an MVCCStats. func (ms *MVCCStatsDelta) ToStats() MVCCStats { @@ -37,6 +37,6 @@ func (ms *MVCCStats) ToPersistentStats() MVCCPersistentStats { func (op *MVCCLogicalOp) MustSetValue(value interface{}) { op.Reset() if !op.SetValue(value) { - panic(fmt.Sprintf("%T excludes %T", op, value)) + panic(errors.AssertionFailedf("%T excludes %T", op, value)) } } diff --git a/pkg/testutils/sort.go b/pkg/testutils/sort.go index 2dc30db1a3d8..c2e9b7097198 100644 --- a/pkg/testutils/sort.go +++ b/pkg/testutils/sort.go @@ -14,6 +14,8 @@ import ( "fmt" "reflect" "sort" + + "github.com/cockroachdb/errors" ) var _ sort.Interface = structSorter{} @@ -106,14 +108,14 @@ func SortStructs(s interface{}, fieldNames ...string) { // Verify that we've gotten a slice of structs or pointers to structs. structs := reflect.ValueOf(s) if structs.Kind() != reflect.Slice { - panic(fmt.Sprintf("expected slice, got %T", s)) + panic(errors.AssertionFailedf("expected slice, got %T", s)) } elemType := structs.Type().Elem() if elemType.Kind() == reflect.Ptr { elemType = elemType.Elem() } if elemType.Kind() != reflect.Struct { - panic(fmt.Sprintf("%s is not a struct or pointer to struct", structs.Elem())) + panic(errors.AssertionFailedf("%s is not a struct or pointer to struct", structs.Elem())) } sort.Sort(structSorter{structs, fieldNames}) diff --git a/pkg/util/log/redact_test.go b/pkg/util/log/redact_test.go index c52ab9831201..2ac99d356031 100644 --- a/pkg/util/log/redact_test.go +++ b/pkg/util/log/redact_test.go @@ -18,6 +18,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" "github.com/cockroachdb/redact" "github.com/stretchr/testify/assert" @@ -62,6 +63,19 @@ func TestRedactedLogOutput(t *testing.T) { if !contains("test3 "+startRedactable+"hello"+endRedactable+" end", t) { t.Errorf("expected marked data, got %q", contents()) } + + // Verify that safe parts of errors don't get enclosed in redaction markers + mainLog.newBuffers() + Errorf(context.Background(), "test3e %v end", + errors.AssertionFailedf("hello %v", + errors.Newf("error-in-error %s", "world"))) + if !contains(redactableIndicator+" test3e", t) { + t.Errorf("expected marker indicator, got %q", contents()) + } + if !contains("test3e hello error-in-error "+startRedactable+"world"+endRedactable+" end", t) { + t.Errorf("expected marked data, got %q", contents()) + } + // When redactable logs are enabled, the markers are always quoted. mainLog.newBuffers() const specialString = "x" + startRedactable + "hello" + endRedactable + "y" @@ -117,9 +131,9 @@ func TestRedactedDecodeFile(t *testing.T) { {false, WithMarkedSensitiveData, true, "‹marker: this is safe, stray marks ??, this is not safe›"}, {false, WithFlattenedSensitiveData, false, "marker: this is safe, stray marks ‹›, this is not safe"}, {false, WithoutSensitiveData, true, "‹×›"}, - {true, WithMarkedSensitiveData, true, "marker: this is safe, stray marks ‹›, ‹this is not safe›"}, - {true, WithFlattenedSensitiveData, false, "marker: this is safe, stray marks , this is not safe"}, - {true, WithoutSensitiveData, true, "marker: this is safe, stray marks ‹×›, ‹×›"}, + {true, WithMarkedSensitiveData, true, "marker: this is safe, stray marks ??, ‹this is not safe›"}, + {true, WithFlattenedSensitiveData, false, "marker: this is safe, stray marks ??, this is not safe"}, + {true, WithoutSensitiveData, true, "marker: this is safe, stray marks ??, ‹×›"}, } for _, tc := range testData { diff --git a/pkg/util/protoutil/clone.go b/pkg/util/protoutil/clone.go index 06be12995eec..5fc298490a39 100644 --- a/pkg/util/protoutil/clone.go +++ b/pkg/util/protoutil/clone.go @@ -11,10 +11,10 @@ package protoutil import ( - "fmt" "reflect" "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" "github.com/gogo/protobuf/proto" ) @@ -74,7 +74,7 @@ func uncloneable(pb Message) (reflect.Type, bool) { // upstream, see https://github.com/gogo/protobuf/issues/147. func Clone(pb Message) Message { if t, ok := uncloneable(pb); ok { - panic(fmt.Sprintf("attempt to clone %T, which contains uncloneable field of type %s", pb, t)) + panic(errors.AssertionFailedf("attempt to clone %T, which contains uncloneable field of type %s", pb, t)) } return proto.Clone(pb).(Message) } diff --git a/pkg/workload/querylog/querylog.go b/pkg/workload/querylog/querylog.go index aee7194bbcac..841c1ae6e2d8 100644 --- a/pkg/workload/querylog/querylog.go +++ b/pkg/workload/querylog/querylog.go @@ -991,7 +991,7 @@ func printPlaceholder(i interface{}) string { case nil: return fmt.Sprintf("NULL") default: - panic(fmt.Sprintf("unsupported type: %T", i)) + panic(errors.AssertionFailedf("unsupported type: %T", i)) } } diff --git a/pkg/workload/workload.go b/pkg/workload/workload.go index 12dba8317157..514aa626b45b 100644 --- a/pkg/workload/workload.go +++ b/pkg/workload/workload.go @@ -239,7 +239,7 @@ func TypedTuples(count int, typs []*types.T, fn func(int) []interface{}) Batched case time.Time: col.Bytes().Set(0, []byte(d.Round(time.Microsecond).UTC().Format(timestampOutputFormat))) default: - panic(fmt.Sprintf(`unhandled datum type %T`, d)) + panic(errors.AssertionFailedf(`unhandled datum type %T`, d)) } } } @@ -458,6 +458,6 @@ func ApproxDatumSize(x interface{}) int64 { case time.Time: return 12 default: - panic(fmt.Sprintf("unsupported type %T: %v", x, x)) + panic(errors.AssertionFailedf("unsupported type %T: %v", x, x)) } } diff --git a/pkg/workload/workloadsql/workloadsql.go b/pkg/workload/workloadsql/workloadsql.go index f32da770f835..15e3d7e89188 100644 --- a/pkg/workload/workloadsql/workloadsql.go +++ b/pkg/workload/workloadsql/workloadsql.go @@ -214,7 +214,7 @@ func StringTuple(datums []interface{}) []string { // See the HACK comment in ColBatchToRows. s[i] = lex.EscapeSQLString(string(x)) default: - panic(fmt.Sprintf("unsupported type %T: %v", x, x)) + panic(errors.AssertionFailedf("unsupported type %T: %v", x, x)) } } return s @@ -265,7 +265,7 @@ func (s sliceSliceInterface) Less(i, j int) bool { case []byte: cmp = bytes.Compare(x, s[j][offset].([]byte)) default: - panic(fmt.Sprintf("unsupported type %T: %v", x, x)) + panic(errors.AssertionFailedf("unsupported type %T: %v", x, x)) } if cmp < 0 { return true diff --git a/vendor b/vendor index e45b0079e30c..49c7fe7b4405 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit e45b0079e30c9a7fc384525896c9d5d4b288f793 +Subproject commit 49c7fe7b4405b33447f29e1c40d7a1bb959d1bf4