Skip to content

Commit

Permalink
pgwire: add telemetry for fetch limits
Browse files Browse the repository at this point in the history
The JDBC driver and perhaps others commonly try to use the "fetch
limit" parameter, which is yet unsupported in
CockroachDB (#4035). This patch adds telemetry to gauge demand.

Release note (sql change): attempts by client apps to use the
unsupported "fetch limit" parameter (e.g. via JDBC) will now be
captured in telemetry if statistics reporting is enabled, to gauge
support for this feature.
  • Loading branch information
knz committed Oct 23, 2018
1 parent a4fe47a commit 5e189d5
Show file tree
Hide file tree
Showing 4 changed files with 45 additions and 39 deletions.
35 changes: 35 additions & 0 deletions pkg/server/telemetry/features.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@ import (
"fmt"
"sync/atomic"

"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
)

Expand Down Expand Up @@ -123,3 +125,36 @@ func GetAndResetFeatureCounts(quantize bool) map[string]int32 {
}
return m
}

// RecordError takes an error and increments the corresponding count
// for its error code, and, if it is an unimplemented or internal
// error, the count for that feature or the internal error's shortened
// stack trace.
func RecordError(err error) {
if err == nil {
return
}

if pgErr, ok := pgerror.GetPGCause(err); ok {
Count("errorcodes." + pgErr.Code)

if details := pgErr.InternalCommand; details != "" {
var prefix string
switch pgErr.Code {
case pgerror.CodeFeatureNotSupportedError:
prefix = "unimplemented."
case pgerror.CodeInternalError:
prefix = "internalerror."
default:
prefix = "othererror." + pgErr.Code + "."
}
Count(prefix + details)
}
} else {
typ := log.ErrorSource(err)
if typ == "" {
typ = "unknown"
}
Count("othererror." + typ)
}
}
37 changes: 2 additions & 35 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -302,39 +302,6 @@ func (s *Server) Start(ctx context.Context, stopper *stop.Stopper) {
s.PeriodicallyClearStmtStats(ctx, stopper)
}

// recordError takes an error and increments the corresponding count
// for its error code, and, if it is an unimplemented or internal
// error, the count for that feature or the internal error's shortened
// stack trace.
func (s *Server) recordError(err error) {
if err == nil {
return
}

if pgErr, ok := pgerror.GetPGCause(err); ok {
telemetry.Count("errorcodes." + pgErr.Code)

if details := pgErr.InternalCommand; details != "" {
var prefix string
switch pgErr.Code {
case pgerror.CodeFeatureNotSupportedError:
prefix = "unimplemented."
case pgerror.CodeInternalError:
prefix = "internalerror."
default:
prefix = "othererror." + pgErr.Code + "."
}
telemetry.Count(prefix + details)
}
} else {
typ := log.ErrorSource(err)
if typ == "" {
typ = "unknown"
}
telemetry.Count("othererror." + typ)
}
}

// ResetStatementStats resets the executor's collected statement statistics.
func (s *Server) ResetStatementStats(ctx context.Context) {
s.sqlStats.resetStats(ctx)
Expand Down Expand Up @@ -1292,12 +1259,12 @@ func (ex *connExecutor) run(
ex.sessionEventf(ex.Ctx(), "execution error: %s", pe.errorCause())
}
if resErr == nil && ok {
ex.server.recordError(pe.errorCause())
telemetry.RecordError(pe.errorCause())
// Depending on whether the result has the error already or not, we have
// to call either Close or CloseWithErr.
res.CloseWithErr(pe.errorCause())
} else {
ex.server.recordError(resErr)
telemetry.RecordError(resErr)
res.Close(stateToTxnStatusIndicator(ex.machine.CurState()))
}
} else {
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/coltypes"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -1207,7 +1208,7 @@ func (ex *connExecutor) runShowSyntax(
commErr = res.AddRow(ctx, tree.Datums{tree.NewDString(field), tree.NewDString(msg)})
return nil
},
ex.server.recordError, /* reportErr */
telemetry.RecordError, /* reportErr */
); err != nil {
res.SetError(err)
}
Expand Down
9 changes: 6 additions & 3 deletions pkg/sql/pgwire/command_result.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,14 @@ import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/lib/pq/oid"
"github.com/pkg/errors"
)

type completionMsgType int
Expand Down Expand Up @@ -124,9 +125,11 @@ func (r *commandResult) Close(t sql.TransactionStatusIndicator) {
r.typ == commandComplete &&
r.stmtType == tree.Rows {

r.err = errors.Errorf("execute row count limits not supported: %d of %d",
r.err = pgerror.UnimplementedWithIssueErrorf(4035,
"execute row count limits not supported: %d of %d",
r.limit, r.rowsAffected)
r.conn.bufferErr(convertToErrWithPGCode(r.err))
telemetry.RecordError(r.err)
r.conn.bufferErr(r.err)
}

// Send a completion message, specific to the type of result.
Expand Down

0 comments on commit 5e189d5

Please sign in to comment.