Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
…db#86748 cockroachdb#86809

86608: batcheval: add latch key protecting range key stats update r=erikgrinaker a=aliher1911

Previously GC needed to get a read latch with max timestamp to
ensure that range tombstones are not modified during GC. This
is causing all writers to get stuck in queue while GC is validating
request and removing range tombstone.
This commit adds a dedicated latch key
LocalRangeRangeTombstoneStatsUpdateLockSuffix to address the problem.
All range tombstone writers obtain this read latch on top of the write
latches for the ranges they are interested to update.
GC on the other hand will obtain write latch on that key. This
approach allows point writers to proceed during GC, but will block new
range tombstones from being written. Non conflicting writes of range
tombstones could still proceed since their write latch ranges don't
overlap.

Release justification: this is a safe change as range tombstone
behaviour is not enabled yet and the change is needed to address
potential performance regressions.

Release note: None

86645: kvserver: log when raft send/recv queue fills up r=pavelkalinnikov a=tbg

Inspired by https://github.com/cockroachlabs/support/issues/1770.

If either the raft send or receive queue fills up, wide-spread outages
can occur as replication progress stalls. We have metrics that can
indicate this, but straightforward logging is also appropriate to direct
attention to the fact, which this commit achieves.

Touches cockroachdb#79755

Release justification: important logging improvement
Release note: None


86679: server,ui: show internal stats with new cluster setting r=maryliag a=maryliag

Previously, we were not showing internal results on
fingerprint options on SQL Activity.
A new cluster setting created `sql.stats.response.show_internal`
can be set to `true` and internal statistics will be
displayed on SQL Activity page.

Fixes cockroachdb#79547

https://www.loom.com/share/1b89ba99a7c247edadb5c8b0d127755c

Release justification: low risk, high benefit change
Release note (sql change): New cluster setting
`sql.stats.response.show_internal` with default value of `false`
can be set to true, to display information about internal stats
on SQL Activity page, with fingerprint option.

86748: storage: rename `MVCCRangeKeyStack.FirstAbove/Below` r=tbg a=erikgrinaker

This patch renames `FirstAbove/Below` to `FirstAtOrAbove/Below`, for
clarity.

Release justification: bug fixes and low-risk updates to new functionality

Release note: None

86809: backupccl: set kv.bulkio.write_metadata_sst.enabled to default false r=stevendanna a=msbutler

This patch sets write_metadata_sst cluster setting to false in prep for the
22.2 branch cut, as there's additional worked required before this feature gets
used in production.

Setting this to false will also stop the roachtest in cockroachdb#86289 from consistently
failing due to cockroachdb#86806.

Fixes cockroachdb#86289

Release note: none

Release justification: prevents using an experimental feature by default

Co-authored-by: Oleg Afanasyev <[email protected]>
Co-authored-by: Tobias Grieger <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Michael Butler <[email protected]>
  • Loading branch information
6 people committed Aug 25, 2022
6 parents 9a7ec4b + d15851a + c1cd955 + 3750821 + 9fb1fb7 + f74435a commit 54bc65f
Show file tree
Hide file tree
Showing 25 changed files with 189 additions and 80 deletions.
1 change: 1 addition & 0 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,7 @@ sql.stats.non_default_columns.min_retention_period duration 24h0m0s minimum rete
sql.stats.persisted_rows.max integer 1000000 maximum number of rows of statement and transaction statistics that will be persisted in the system tables
sql.stats.post_events.enabled boolean false if set, an event is logged for every CREATE STATISTICS job
sql.stats.response.max integer 20000 the maximum number of statements and transaction stats returned in a CombinedStatements request
sql.stats.response.show_internal.enabled boolean false controls if statistics for internal executions should be returned by the CombinedStatements endpoint. This endpoint is used to display statistics on the Statement and Transaction fingerprint pages under SQL Activity
sql.stats.system_tables.enabled boolean true when true, enables use of statistics on system tables by the query optimizer
sql.stats.system_tables_autostats.enabled boolean true when true, enables automatic collection of statistics on system tables
sql.telemetry.query_sampling.enabled boolean false when set to true, executed queries will emit an event on the telemetry logging channel
Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -203,6 +203,7 @@
<tr><td><code>sql.stats.persisted_rows.max</code></td><td>integer</td><td><code>1000000</code></td><td>maximum number of rows of statement and transaction statistics that will be persisted in the system tables</td></tr>
<tr><td><code>sql.stats.post_events.enabled</code></td><td>boolean</td><td><code>false</code></td><td>if set, an event is logged for every CREATE STATISTICS job</td></tr>
<tr><td><code>sql.stats.response.max</code></td><td>integer</td><td><code>20000</code></td><td>the maximum number of statements and transaction stats returned in a CombinedStatements request</td></tr>
<tr><td><code>sql.stats.response.show_internal.enabled</code></td><td>boolean</td><td><code>false</code></td><td>controls if statistics for internal executions should be returned by the CombinedStatements endpoint. This endpoint is used to display statistics on the Statement and Transaction fingerprint pages under SQL Activity</td></tr>
<tr><td><code>sql.stats.system_tables.enabled</code></td><td>boolean</td><td><code>true</code></td><td>when true, enables use of statistics on system tables by the query optimizer</td></tr>
<tr><td><code>sql.stats.system_tables_autostats.enabled</code></td><td>boolean</td><td><code>true</code></td><td>when true, enables automatic collection of statistics on system tables</td></tr>
<tr><td><code>sql.telemetry.query_sampling.enabled</code></td><td>boolean</td><td><code>false</code></td><td>when set to true, executed queries will emit an event on the telemetry logging channel</td></tr>
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/backupinfo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/stats",
"//pkg/storage",
"//pkg/util",
"//pkg/util/ctxgroup",
"//pkg/util/encoding",
"//pkg/util/hlc",
Expand Down
5 changes: 2 additions & 3 deletions pkg/ccl/backupccl/backupinfo/manifest_handling.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -82,13 +83,11 @@ const (

// WriteMetadataSST controls if we write the experimental new format BACKUP
// metadata file.
// kv.bulkio.rite_metadata_sst.enabled set to false by default due to
// https://github.com/cockroachdb/cockroach/issues/85564.
var WriteMetadataSST = settings.RegisterBoolSetting(
settings.TenantWritable,
"kv.bulkio.write_metadata_sst.enabled",
"write experimental new format BACKUP metadata file",
true,
util.ConstantWithMetamorphicTestBool("write-metadata-sst", false),
)

// IsGZipped detects whether the given bytes represent GZipped data. This check
Expand Down
4 changes: 4 additions & 0 deletions pkg/keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,10 @@ var (
// LocalRangeLastReplicaGCTimestampSuffix is the suffix for a range's last
// replica GC timestamp (for GC of old replicas).
LocalRangeLastReplicaGCTimestampSuffix = []byte("rlrt")
// LocalRangeMVCCRangeKeyGCLockSuffix is the suffix for a lock obtained
// by range tombstone operations to ensure they don't overlap with
// GC requests while allowing point traffic to go through unobstructed.
LocalRangeMVCCRangeKeyGCLockSuffix = []byte("rltu")
// localRangeLastVerificationTimestampSuffix is DEPRECATED and remains to
// prevent reuse.
localRangeLastVerificationTimestampSuffix = []byte("rlvt")
Expand Down
12 changes: 12 additions & 0 deletions pkg/keys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,12 @@ func RangeGCThresholdKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RangeGCThresholdKey()
}

// MVCCRangeKeyGCKey returns a range local key protecting range
// tombstone mvcc stats calculations during range tombstone GC.
func MVCCRangeKeyGCKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).MVCCRangeKeyGCKey()
}

// RangeVersionKey returns a system-local for the range version.
func RangeVersionKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RangeVersionKey()
Expand Down Expand Up @@ -1055,3 +1061,9 @@ func (b RangeIDPrefixBuf) RaftReplicaIDKey() roachpb.Key {
func (b RangeIDPrefixBuf) RangeLastReplicaGCTimestampKey() roachpb.Key {
return append(b.unreplicatedPrefix(), LocalRangeLastReplicaGCTimestampSuffix...)
}

// MVCCRangeKeyGCKey returns a range local key protecting range
// tombstone mvcc stats calculations during range tombstone GC.
func (b RangeIDPrefixBuf) MVCCRangeKeyGCKey() roachpb.Key {
return append(b.unreplicatedPrefix(), LocalRangeMVCCRangeKeyGCLockSuffix...)
}
9 changes: 9 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,8 +57,17 @@ func declareKeysAddSSTable(
// NB: The range end key is not available, so this will pessimistically
// latch up to args.EndKey.Next(). If EndKey falls on the range end key, the
// span will be tightened during evaluation.
// Even if we obtain latches beyond the end range here, it won't cause
// contention with the subsequent range because latches are enforced per
// range.
l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil)
latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp)

// Obtain a read only lock on range key GC key to serialize with
// range tombstone GC requests.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()),
})
}
}

Expand Down
9 changes: 9 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_clear_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,9 +56,18 @@ func declareKeysClearRange(
// NB: The range end key is not available, so this will pessimistically latch
// up to args.EndKey.Next(). If EndKey falls on the range end key, the span
// will be tightened during evaluation.
// Even if we obtain latches beyond the end range here, it won't cause
// contention with the subsequent range because latches are enforced per
// range.
args := req.(*roachpb.ClearRangeRequest)
l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil)
latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp)

// Obtain a read only lock on range key GC key to serialize with
// range tombstone GC requests.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()),
})
}

// ClearRange wipes all MVCC versions of keys covered by the specified
Expand Down
9 changes: 9 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,13 +49,22 @@ func declareKeysDeleteRange(
// NB: The range end key is not available, so this will pessimistically
// latch up to args.EndKey.Next(). If EndKey falls on the range end key, the
// span will be tightened during evaluation.
// Even if we obtain latches beyond the end range here, it won't cause
// contention with the subsequent range because latches are enforced per
// range.
l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil)
latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp)

// We need to read the range descriptor to determine the bounds during eval.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.RangeDescriptorKey(rs.GetStartKey()),
})

// Obtain a read only lock on range key GC key to serialize with
// range tombstone GC requests.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()),
})
}
}

Expand Down
11 changes: 11 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,12 @@ func declareKeysEndTxn(
Key: abortspan.MinKey(rs.GetRangeID()),
EndKey: abortspan.MaxKey(rs.GetRangeID()),
})

// Protect range tombstones from collection by GC to avoid interference
// with MVCCStats calculation.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()),
})
}
if mt := et.InternalCommitTrigger.MergeTrigger; mt != nil {
// Merges copy over the RHS abort span to the LHS, and compute
Expand Down Expand Up @@ -183,6 +189,11 @@ func declareKeysEndTxn(
Key: leftPeekBound,
EndKey: rightPeekBound,
})
// Protect range tombstones from collection by GC to avoid interference
// with MVCCStats calculation.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.MVCCRangeKeyGCKey(mt.LeftDesc.RangeID),
})
}
}
}
Expand Down
33 changes: 18 additions & 15 deletions pkg/kv/kvserver/batcheval/cmd_gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,22 +37,25 @@ func declareKeysGC(
_ time.Duration,
) {
gcr := req.(*roachpb.GCRequest)
// When GC-ing MVCC range key tombstones or individual range keys, we need to
// serialize with all writes that overlap the MVCC range tombstone, as well as
// the immediate left/right neighboring keys. This is because a range key
// write has non-local effects, i.e. it can fragment or merge other range keys
// at other timestamps and at its boundaries, and this has a non-commutative
// effect on MVCC stats -- if someone writes a new range key while we're GCing
// one below, the stats would come out wrong.
// Note that we only need to serialize with writers (including other GC
// processes) and not with readers (that are guaranteed to be above the GC
// threshold). To achieve this, we declare read-write access at
// hlc.MaxTimestamp which will not block any readers.
for _, span := range mergeAdjacentSpans(makeLookupBoundariesForGCRanges(
rs.GetStartKey().AsRawKey(), nil, gcr.RangeKeys,
)) {
latchSpans.AddMVCC(spanset.SpanReadWrite, span, hlc.MaxTimestamp)
if gcr.RangeKeys != nil {
// When GC-ing MVCC range key tombstones, we need to serialize with
// range key writes that overlap the MVCC range tombstone, as well as
// the immediate left/right neighboring keys. This is because a range
// key write has non-local effects, i.e. it can fragment or merge other
// range keys at other timestamps and at its boundaries, and this has
// a non-commutative effect on MVCC stats -- if someone writes a new
// range key while we're GCing one below, the stats would come out wrong.
//
// To achieve this, we use a virtual latch key that will prevent any
// range key write operations until GC request is done. All other range
// tombstone write operations obtain a read latch on this key and can
// run concurrently.
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()),
})
}
// For ClearRangeKey request we still obtain a wide write lock as we don't
// expect any operations running on the range.
if rk := gcr.ClearRangeKey; rk != nil {
latchSpans.AddMVCC(spanset.SpanReadWrite, roachpb.Span{Key: rk.StartKey, EndKey: rk.EndKey},
hlc.MaxTimestamp)
Expand Down
9 changes: 9 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_revert_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,8 +54,17 @@ func declareKeysRevertRange(
// NB: The range end key is not available, so this will pessimistically
// latch up to args.EndKey.Next(). If EndKey falls on the range end key, the
// span will be tightened during evaluation.
// Even if we obtain latches beyond the end range here, it won't cause
// contention with the subsequent range because latches are enforced per
// range.
l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil)
latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp)

// Obtain a read only lock on range key GC key to serialize with
// range tombstone GC requests.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()),
})
}

// isEmptyKeyTimeRange checks if the span has no writes in (since,until].
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/gc/gc_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ func (it *gcIterator) currentRangeTS() hlc.Timestamp {
}
it.cachedRangeTombstoneKey = append(it.cachedRangeTombstoneKey[:0], rangeTombstoneStartKey...)

if v, ok := it.it.RangeKeys().FirstBelow(it.threshold); ok {
if v, ok := it.it.RangeKeys().FirstAtOrBelow(it.threshold); ok {
it.cachedRangeTombstoneTS = v.Timestamp
} else {
it.cachedRangeTombstoneTS = hlc.Timestamp{}
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/raft_transport.go
Original file line number Diff line number Diff line change
Expand Up @@ -505,6 +505,9 @@ func (t *RaftTransport) SendAsync(
case ch <- req:
return true
default:
if logRaftSendQueueFullEvery.ShouldLog() {
log.Warningf(t.AnnotateCtx(context.Background()), "raft send queue to n%d is full", toNodeID)
}
releaseRaftMessageRequest(req)
return false
}
Expand Down
8 changes: 8 additions & 0 deletions pkg/kv/kvserver/store_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,11 @@ import (
"go.etcd.io/etcd/raft/v3/raftpb"
)

var (
logRaftRecvQueueFullEvery = log.Every(1 * time.Second)
logRaftSendQueueFullEvery = log.Every(1 * time.Second)
)

type raftRequestInfo struct {
req *kvserverpb.RaftMessageRequest
size int64 // size of req in bytes
Expand Down Expand Up @@ -305,6 +310,9 @@ func (s *Store) HandleRaftUncoalescedRequest(
// that dropping the request is safe. Raft will retry.
s.metrics.RaftRcvdDropped.Inc(1)
s.metrics.RaftRcvdDroppedBytes.Inc(size)
if logRaftRecvQueueFullEvery.ShouldLog() {
log.Warningf(ctx, "raft receive queue for r%d is full", req.RangeID)
}
return false
}
return enqueue
Expand Down
10 changes: 10 additions & 0 deletions pkg/server/cluster_settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,3 +21,13 @@ var SQLStatsResponseMax = settings.RegisterIntSetting(
20000,
settings.NonNegativeInt,
).WithPublic()

// SQLStatsShowInternal controls if statistics for internal executions should be returned by the
// CombinedStatements endpoint.
var SQLStatsShowInternal = settings.RegisterBoolSetting(
settings.TenantWritable,
"sql.stats.response.show_internal.enabled",
"controls if statistics for internal executions should be returned by the CombinedStatements endpoint. This "+
"endpoint is used to display statistics on the Statement and Transaction fingerprint pages under SQL Activity",
false,
).WithPublic()
32 changes: 24 additions & 8 deletions pkg/server/combined_statement_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,9 @@ func getCombinedStatementStats(
startTime := getTimeFromSeconds(req.Start)
endTime := getTimeFromSeconds(req.End)
limit := SQLStatsResponseMax.Get(&settings.SV)
whereClause, orderAndLimit, args := getCombinedStatementsQueryClausesAndArgs(startTime, endTime, limit, testingKnobs)
showInternal := SQLStatsShowInternal.Get(&settings.SV)
whereClause, orderAndLimit, args := getCombinedStatementsQueryClausesAndArgs(
startTime, endTime, limit, testingKnobs, showInternal)
statements, err := collectCombinedStatements(ctx, ie, whereClause, args, orderAndLimit)
if err != nil {
return nil, serverError(ctx, err)
Expand Down Expand Up @@ -98,13 +100,17 @@ func getCombinedStatementStats(
// The whereClause will be in the format `WHERE A = $1 AND B = $2` and
// args will return the list of arguments in order that will replace the actual values.
func getCombinedStatementsQueryClausesAndArgs(
start, end *time.Time, limit int64, testingKnobs *sqlstats.TestingKnobs,
start, end *time.Time, limit int64, testingKnobs *sqlstats.TestingKnobs, showInternal bool,
) (whereClause string, orderAndLimitClause string, args []interface{}) {
var buffer strings.Builder
buffer.WriteString(testingKnobs.GetAOSTClause())

// Filter out internal statements by app name.
buffer.WriteString(fmt.Sprintf(" WHERE app_name NOT LIKE '%s%%'", catconstants.InternalAppNamePrefix))
if showInternal {
buffer.WriteString(" WHERE true")
} else {
// Filter out internal statements by app name.
buffer.WriteString(fmt.Sprintf(" WHERE app_name NOT LIKE '%s%%'", catconstants.InternalAppNamePrefix))
}

if start != nil {
buffer.WriteString(" AND aggregated_ts >= $1")
Expand Down Expand Up @@ -357,7 +363,8 @@ func getStatementDetails(
testingKnobs *sqlstats.TestingKnobs,
) (*serverpb.StatementDetailsResponse, error) {
limit := SQLStatsResponseMax.Get(&settings.SV)
whereClause, args, err := getStatementDetailsQueryClausesAndArgs(req, testingKnobs)
showInternal := SQLStatsShowInternal.Get(&settings.SV)
whereClause, args, err := getStatementDetailsQueryClausesAndArgs(req, testingKnobs, showInternal)
if err != nil {
return nil, serverError(ctx, err)
}
Expand Down Expand Up @@ -407,7 +414,7 @@ func getStatementDetails(
// The whereClause will be in the format `WHERE A = $1 AND B = $2` and
// args will return the list of arguments in order that will replace the actual values.
func getStatementDetailsQueryClausesAndArgs(
req *serverpb.StatementDetailsRequest, testingKnobs *sqlstats.TestingKnobs,
req *serverpb.StatementDetailsRequest, testingKnobs *sqlstats.TestingKnobs, showInternal bool,
) (whereClause string, args []interface{}, err error) {
var buffer strings.Builder
buffer.WriteString(testingKnobs.GetAOSTClause())
Expand All @@ -420,18 +427,24 @@ func getStatementDetailsQueryClausesAndArgs(
args = append(args, sqlstatsutil.EncodeUint64ToBytes(fingerprintID))
buffer.WriteString(fmt.Sprintf(" WHERE fingerprint_id = $%d", len(args)))

// Filter out internal statements by app name.
buffer.WriteString(fmt.Sprintf(" AND app_name NOT LIKE '%s%%'", catconstants.InternalAppNamePrefix))
if !showInternal {
// Filter out internal statements by app name.
buffer.WriteString(fmt.Sprintf(" AND app_name NOT LIKE '%s%%'", catconstants.InternalAppNamePrefix))
}

// Statements are grouped ignoring the app name in the Statements/Transactions page, so when
// calling for the Statement Details endpoint, this value can be empty or a list of app names.
if len(req.AppNames) > 0 {
if !(len(req.AppNames) == 1 && req.AppNames[0] == "") {
hasInternal := false
buffer.WriteString(" AND (")
for i, app := range req.AppNames {
if app == "(unset)" {
app = ""
}
if strings.Contains(app, catconstants.InternalAppNamePrefix) {
hasInternal = true
}
if i != 0 {
args = append(args, app)
buffer.WriteString(fmt.Sprintf(" OR app_name = $%d", len(args)))
Expand All @@ -440,6 +453,9 @@ func getStatementDetailsQueryClausesAndArgs(
buffer.WriteString(fmt.Sprintf(" app_name = $%d", len(args)))
}
}
if hasInternal {
buffer.WriteString(fmt.Sprintf(" OR app_name LIKE '%s%%'", catconstants.InternalAppNamePrefix))
}
buffer.WriteString(" )")
}
}
Expand Down
Loading

0 comments on commit 54bc65f

Please sign in to comment.